From c8e8abedbdfec6e92b0c63e90f3c2c5755fd8978 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Fri, 9 Mar 2018 15:39:36 -0800 Subject: [PATCH 01/29] SPARK-23429: Add executor memory metrics to heartbeat and expose in executors REST API Add new executor level memory metrics (JVM used memory, on/off heap execution memory, on/off heap storage memory), and expose via the executors REST API. This information will help provide insight into how executor and driver JVM memory is used, and for the different memory regions. It can be used to help determine good values for spark.executor.memory, spark.driver.memory, spark.memory.fraction, and spark.memory.storageFraction. Add an ExecutorMetrics class, with jvmUsedMemory, onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, and offHeapStorageMemory. The new ExecutorMetrics will be sent by executors to the driver as part of Heartbeat. A heartbeat will be added for the driver as well, to collect these metrics for the driver. Modify the EventLoggingListener to log ExecutorMetricsUpdate events if there is a new peak value for any of the memory metrics for an executor and stage. Only the ExecutorMetrics will be logged, and not the TaskMetrics, to minimize additional logging. Modify the AppStatusListener to record the peak values for each memory metric. Add the new memory metrics to the executors REST API. --- .../org/apache/spark/HeartbeatReceiver.scala | 8 +- .../scala/org/apache/spark/Heartbeater.scala | 52 +++ .../org/apache/spark/executor/Executor.scala | 32 +- .../spark/executor/ExecutorMetrics.scala | 43 ++ .../apache/spark/memory/MemoryManager.scala | 20 + .../apache/spark/scheduler/DAGScheduler.scala | 29 +- .../scheduler/EventLoggingListener.scala | 30 +- .../spark/scheduler/PeakExecutorMetrics.scala | 106 +++++ .../spark/scheduler/SparkListener.scala | 6 +- .../spark/scheduler/TaskScheduler.scala | 10 +- .../spark/scheduler/TaskSchedulerImpl.scala | 13 +- .../spark/status/AppStatusListener.scala | 10 +- .../org/apache/spark/status/LiveEntity.scala | 9 +- .../org/apache/spark/status/api/v1/api.scala | 13 +- .../org/apache/spark/util/JsonProtocol.scala | 43 +- .../application_list_json_expectation.json | 15 + .../completed_app_list_json_expectation.json | 15 + ...ith_executor_metrics_json_expectation.json | 367 ++++++++++++++++++ .../limit_app_list_json_expectation.json | 30 +- .../minDate_app_list_json_expectation.json | 15 + .../minEndDate_app_list_json_expectation.json | 17 +- .../application_1506645932520_24630151 | 106 +++++ .../apache/spark/HeartbeatReceiverSuite.scala | 10 +- .../deploy/history/HistoryServerSuite.scala | 2 + .../spark/scheduler/DAGSchedulerSuite.scala | 7 +- .../scheduler/EventLoggingListenerSuite.scala | 166 ++++++++ .../ExternalClusterManagerSuite.scala | 4 +- .../apache/spark/util/JsonProtocolSuite.scala | 79 +++- dev/.rat-excludes | 1 + 29 files changed, 1186 insertions(+), 72 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/Heartbeater.scala create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala create mode 100644 core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json create mode 100644 core/src/test/resources/spark-events/application_1506645932520_24630151 diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index ff960b396dbf..ac6fb1f2659c 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable import scala.concurrent.Future +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ @@ -37,7 +38,8 @@ import org.apache.spark.util._ private[spark] case class Heartbeat( executorId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates - blockManagerId: BlockManagerId) + blockManagerId: BlockManagerId, + executorUpdates: ExecutorMetrics) // executor level updates /** * An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is @@ -120,14 +122,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) context.reply(true) // Messages received from executors - case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId) => + case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId, executorMetrics) => if (scheduler != null) { if (executorLastSeen.contains(executorId)) { executorLastSeen(executorId) = clock.getTimeMillis() eventLoopThread.submit(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { val unknownExecutor = !scheduler.executorHeartbeatReceived( - executorId, accumUpdates, blockManagerId) + executorId, accumUpdates, blockManagerId, executorMetrics) val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) context.reply(response) } diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala new file mode 100644 index 000000000000..b36285ed42e6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -0,0 +1,52 @@ +/* + * 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 + +import java.util.concurrent.TimeUnit + +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * Creates a heartbeat thread which will call the specified reportHeartbeat function at + * intervals of intervalMs. + * + * @param reportHeartbeat the heartbeat reporting function to call. + * @param intervalMs the interval between heartbeats. + */ +private[spark] class Heartbeater(reportHeartbeat: () => Unit, intervalMs: Long) { + // Executor for the heartbeat task + private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") + + /** Schedules a task to report a heartbeat. */ + private[spark] def start(): Unit = { + // Wait a random interval so the heartbeats don't end up in sync + val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] + + val heartbeatTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(reportHeartbeat()) + } + heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) + } + + /** Stops the heartbeat thread. */ + private[spark] def stop(): Unit = { + heartbeater.shutdown() + heartbeater.awaitTermination(10, TimeUnit.SECONDS) + } +} + diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c325222b764b..16a7c58dade7 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -148,7 +148,8 @@ private[spark] class Executor( private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] // Executor for the heartbeat task. - private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") + private val heartbeater = new Heartbeater(reportHeartBeat, + conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) // must be initialized before running startDriverHeartbeat() private val heartbeatReceiverRef = @@ -167,7 +168,7 @@ private[spark] class Executor( */ private var heartbeatFailures = 0 - startDriverHeartbeater() + heartbeater.start() private[executor] def numRunningTasks: Int = runningTasks.size() @@ -216,8 +217,7 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() - heartbeater.shutdown() - heartbeater.awaitTermination(10, TimeUnit.SECONDS) + heartbeater.stop() threadPool.shutdown() if (!isLocal) { env.stop() @@ -772,6 +772,12 @@ private[spark] class Executor( val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() val curGCTime = computeTotalGcTime() + // get executor level memory metrics + val executorUpdates = new ExecutorMetrics(System.currentTimeMillis(), + ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), + env.memoryManager.onHeapExecutionMemoryUsed, env.memoryManager.offHeapExecutionMemoryUsed, + env.memoryManager.onHeapStorageMemoryUsed, env.memoryManager.offHeapStorageMemoryUsed) + for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { taskRunner.task.metrics.mergeShuffleReadMetrics() @@ -780,7 +786,8 @@ private[spark] class Executor( } } - val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId) + val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId, + executorUpdates) try { val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( message, RpcTimeout(conf, "spark.executor.heartbeatInterval", "10s")) @@ -800,21 +807,6 @@ private[spark] class Executor( } } } - - /** - * Schedules a task to report heartbeat and partial metrics for active tasks to driver. - */ - private def startDriverHeartbeater(): Unit = { - val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") - - // Wait a random interval so the heartbeats don't end up in sync - val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] - - val heartbeatTask = new Runnable() { - override def run(): Unit = Utils.logUncaughtExceptions(reportHeartBeat()) - } - heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) - } } private[spark] object Executor { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala new file mode 100644 index 000000000000..906bdc41d05b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -0,0 +1,43 @@ +/* + * 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.executor + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Executor level metrics. + * + * This is sent to the driver periodically (on executor heartbeat), to provide + * information about each executor's metrics. + * + * @param timestamp the time the metrics were collected + * @param jvmUsedMemory the amount of JVM used memory for the executor + * @param onHeapExecutionMemory the amount of on heap execution memory used + * @param offHeapExecutionMemory the amount of off heap execution memory used + * @param onHeapStorageMemory the amount of on heap storage memory used + * @param offHeapStorageMemory the amount of off heap storage memory used + */ +@DeveloperApi +class ExecutorMetrics private[spark] ( + val timestamp: Long, + val jvmUsedMemory: Long, + val onHeapExecutionMemory: Long, + val offHeapExecutionMemory: Long, + val onHeapStorageMemory: Long, + val offHeapStorageMemory: Long) extends Serializable diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 0641adc2ab69..965515b8abe1 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -180,6 +180,26 @@ private[spark] abstract class MemoryManager( onHeapStorageMemoryPool.memoryUsed + offHeapStorageMemoryPool.memoryUsed } + /** + * On heap execution memory currently in use, in bytes. + */ + final def onHeapExecutionMemoryUsed: Long = onHeapExecutionMemoryPool.memoryUsed + + /** + * Off heap execution memory currently in use, in bytes. + */ + final def offHeapExecutionMemoryUsed: Long = offHeapExecutionMemoryPool.memoryUsed + + /** + * On heap storage memory currently in use, in bytes. + */ + final def onHeapStorageMemoryUsed: Long = onHeapStorageMemoryPool.memoryUsed + + /** + * Off heap storage memory currently in use, in bytes. + */ + final def offHeapStorageMemoryUsed: Long = offHeapStorageMemoryPool.memoryUsed + /** * Returns the execution memory consumption, in bytes, for the given task. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 78b6b34b5d2b..6ad7eaae162c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.NotSerializableException +import java.lang.management.ManagementFactory import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger @@ -34,7 +35,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils @@ -209,6 +210,10 @@ class DAGScheduler( private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) + /** driver heartbeat for collecting metrics */ + private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, + sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + /** * Called by the TaskSetManager to report task's starting. */ @@ -246,8 +251,10 @@ class DAGScheduler( execId: String, // (taskId, stageId, stageAttemptId, accumUpdates) accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], - blockManagerId: BlockManagerId): Boolean = { - listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates)) + blockManagerId: BlockManagerId, + executorUpdates: ExecutorMetrics): Boolean = { + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, + Some(executorUpdates))) blockManagerMaster.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) } @@ -1753,9 +1760,25 @@ class DAGScheduler( messageScheduler.shutdownNow() eventProcessLoop.stop() taskScheduler.stop() + heartbeater.stop() + } + + /** Reports heartbeat metrics for the driver. */ + private def reportHeartBeat(): Unit = { + // get driver memory metrics + val driverUpdates = new ExecutorMetrics(System.currentTimeMillis(), + ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), + sc.env.memoryManager.onHeapExecutionMemoryUsed, + sc.env.memoryManager.offHeapExecutionMemoryUsed, + sc.env.memoryManager.onHeapStorageMemoryUsed, + sc.env.memoryManager.offHeapStorageMemoryUsed) + val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) + listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, + Some(driverUpdates))) } eventProcessLoop.start() + heartbeater.start() } private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 69bc51c1ecf9..e159421f4fbb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -36,6 +36,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.io.CompressionCodec @@ -93,6 +94,9 @@ private[spark] class EventLoggingListener( // Visible for tests only. private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) + // Peak metric values for each executor + private var peakExecutorMetrics = new mutable.HashMap[String, PeakExecutorMetrics]() + /** * Creates the log file in the configured log directory. */ @@ -155,7 +159,11 @@ private[spark] class EventLoggingListener( } // Events that do not trigger a flush - override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = logEvent(event) + override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { + logEvent(event) + // clear the peak metrics when a new stage starts + peakExecutorMetrics.values.foreach(_.reset()) + } override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) @@ -197,10 +205,12 @@ private[spark] class EventLoggingListener( } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { logEvent(event, flushLogger = true) + peakExecutorMetrics.put(event.executorId, new PeakExecutorMetrics()) } override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { logEvent(event, flushLogger = true) + peakExecutorMetrics.remove(event.executorId) } override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { @@ -234,8 +244,22 @@ private[spark] class EventLoggingListener( } } - // No-op because logging every update would be overkill - override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { } + /** + * Log if there is a new peak value for one of the memory metrics for the given executor. + * Metrics are cleared out when a new stage is started in onStageSubmitted, so this will + * log new peak memory metric values per executor per stage. + */ + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { + var log: Boolean = false + event.executorUpdates.foreach { executorUpdates => + val peakMetrics = peakExecutorMetrics.getOrElseUpdate(event.execId, new PeakExecutorMetrics()) + if (peakMetrics.compareAndUpdate(executorUpdates)) { + val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() + logEvent(new SparkListenerExecutorMetricsUpdate(event.execId, accumUpdates, + event.executorUpdates), flushLogger = true) + } + } + } override def onOtherEvent(event: SparkListenerEvent): Unit = { if (event.logEvent) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala new file mode 100644 index 000000000000..187cc77751e6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -0,0 +1,106 @@ +/* + * 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.scheduler + +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.status.api.v1.PeakMemoryMetrics + +/** + * Records the peak values for executor level metrics. If jvmUsedMemory is -1, then no values have + * been recorded yet. + */ +private[spark] class PeakExecutorMetrics { + private var jvmUsedMemory = -1L; + private var onHeapExecutionMemory = 0L + private var offHeapExecutionMemory = 0L + private var onHeapStorageMemory = 0L + private var offHeapStorageMemory = 0L + private var onHeapUnifiedMemory = 0L + private var offHeapUnifiedMemory = 0L + + /** + * Compare the specified memory values with the saved peak executor memory + * values, and update if there is a new peak value. + * + * @param executorMetrics the executor metrics to compare + * @return if there is a new peak value for any metric + */ + def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { + var updated: Boolean = false + + if (executorMetrics.jvmUsedMemory > jvmUsedMemory) { + jvmUsedMemory = executorMetrics.jvmUsedMemory + updated = true + } + if (executorMetrics.onHeapExecutionMemory > onHeapExecutionMemory) { + onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory + updated = true + } + if (executorMetrics.offHeapExecutionMemory > offHeapExecutionMemory) { + offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory + updated = true + } + if (executorMetrics.onHeapStorageMemory > onHeapStorageMemory) { + onHeapStorageMemory = executorMetrics.onHeapStorageMemory + updated = true + } + if (executorMetrics.offHeapStorageMemory > offHeapStorageMemory) { + offHeapStorageMemory = executorMetrics.offHeapStorageMemory + updated = true + } + val newOnHeapUnifiedMemory = (executorMetrics.onHeapExecutionMemory + + executorMetrics.onHeapStorageMemory) + if (newOnHeapUnifiedMemory > onHeapUnifiedMemory) { + onHeapUnifiedMemory = newOnHeapUnifiedMemory + updated = true + } + val newOffHeapUnifiedMemory = (executorMetrics.offHeapExecutionMemory + + executorMetrics.offHeapStorageMemory) + if ( newOffHeapUnifiedMemory > offHeapUnifiedMemory) { + offHeapUnifiedMemory = newOffHeapUnifiedMemory + updated = true + } + + updated + } + + /** + * @return None if no peak metrics have been recorded, else PeakMemoryMetrics with the peak + * values set. + */ + def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = { + if (jvmUsedMemory < 0) { + None + } else { + Some(new PeakMemoryMetrics(jvmUsedMemory, onHeapExecutionMemory, + offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory, + onHeapUnifiedMemory, offHeapUnifiedMemory)) + } + } + + /** Clears/resets the saved peak values. */ + def reset(): Unit = { + jvmUsedMemory = -1L; + onHeapExecutionMemory = 0L + offHeapExecutionMemory = 0L + onHeapStorageMemory = 0L + offHeapStorageMemory = 0L + onHeapUnifiedMemory = 0L + offHeapUnifiedMemory = 0L + } +} 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 8a112f6a37b9..effc0fa53e7c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -26,7 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo import org.apache.spark.{SparkConf, TaskEndReason} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.ui.SparkUI @@ -160,11 +160,13 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends * Periodic updates from executors. * @param execId executor id * @param accumUpdates sequence of (taskId, stageId, stageAttemptId, accumUpdates) + * @param executorUpdates executor level metrics updates */ @DeveloperApi case class SparkListenerExecutorMetricsUpdate( execId: String, - accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])]) + accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])], + executorUpdates: Option[ExecutorMetrics] = None) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 90644fea23ab..037e083c4370 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -68,14 +69,15 @@ private[spark] trait TaskScheduler { def defaultParallelism(): Int /** - * Update metrics for in-progress tasks and let the master know that the BlockManager is still - * alive. Return true if the driver knows about the given block manager. Otherwise, return false, - * indicating that the block manager should re-register. + * Update metrics for in-progress tasks and executor metrics, and let the master know that the + * BlockManager is still alive. Return true if the driver knows about the given block manager. + * Otherwise, return false, indicating that the block manager should re-register. */ def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean + blockManagerId: BlockManagerId, + executorUpdates: ExecutorMetrics): Boolean /** * Get an application ID associated with the job. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 0c11806b3981..b58e66d39027 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -28,6 +28,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -435,14 +436,15 @@ private[spark] class TaskSchedulerImpl( } /** - * Update metrics for in-progress tasks and let the master know that the BlockManager is still - * alive. Return true if the driver knows about the given block manager. Otherwise, return false, - * indicating that the block manager should re-register. + * Update metrics for in-progress tasks and executor metrics, and let the master know that the + * BlockManager is still alive. Return true if the driver knows about the given block manager. + * Otherwise, return false, indicating that the block manager should re-register. */ override def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean = { + blockManagerId: BlockManagerId, + executorMetrics: ExecutorMetrics): Boolean = { // (taskId, stageId, stageAttemptId, accumUpdates) val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { accumUpdates.flatMap { case (id, updates) => @@ -452,7 +454,8 @@ private[spark] class TaskSchedulerImpl( } } } - dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId) + dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId, + executorMetrics) } def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized { 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 5ea161cd0d15..6a54a589b4f7 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1 @@ -669,6 +669,14 @@ private[spark] class AppStatusListener( } } } + event.executorUpdates.foreach { updates: ExecutorMetrics => + // check if there is a new peak value for any of the executor level memory metrics + liveExecutors.get(event.execId).foreach { exec: LiveExecutor => + if (exec.peakExecutorMetrics.compareAndUpdate(updates)) { + maybeUpdate(exec, now) + } + } + } } override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = { 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 79e3f13b826c..afb3e60e0b28 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -27,7 +27,7 @@ import com.google.common.collect.Interners import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, TaskInfo} +import org.apache.spark.scheduler.{AccumulableInfo, PeakExecutorMetrics, StageInfo, TaskInfo} import org.apache.spark.status.api.v1 import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.SparkUI @@ -268,6 +268,9 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE def hasMemoryInfo: Boolean = totalOnHeap >= 0L + // peak values for executor level metrics + var peakExecutorMetrics = new PeakExecutorMetrics + def hostname: String = if (host != null) host else hostPort.split(":")(0) override protected def doUpdate(): Any = { @@ -302,10 +305,10 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE Option(removeReason), executorLogs, memoryMetrics, - blacklistedInStages) + blacklistedInStages, + peakExecutorMetrics.getPeakMemoryMetrics) new ExecutorSummaryWrapper(info) } - } private class LiveExecutorStageSummary( 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 971d7e90fa7b..4d14ba7560e8 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 @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.JobExecutionStatus +import org.apache.spark.executor.ExecutorMetrics case class ApplicationInfo private[spark]( id: String, @@ -98,7 +99,8 @@ class ExecutorSummary private[spark]( val removeReason: Option[String], val executorLogs: Map[String, String], val memoryMetrics: Option[MemoryMetrics], - val blacklistedInStages: Set[Int]) + val blacklistedInStages: Set[Int], + val peakMemoryMetrics: Option[PeakMemoryMetrics]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, @@ -106,6 +108,15 @@ class MemoryMetrics private[spark]( val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) +class PeakMemoryMetrics private[spark]( + val jvmUsedMemory: Long, + val onHeapExecutionMemory: Long, + val offHeapExecutionMemory: Long, + val onHeapStorageMemory: Long, + val offHeapStorageMemory: Long, + val onHeapUnifiedMemory: Long, + val offHeapUnifiedMemory: Long) + class JobData private[spark]( val jobId: Int, val name: String, 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 40383fe05026..18f5275b0af7 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -236,6 +236,7 @@ private[spark] object JsonProtocol { def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = { val execId = metricsUpdate.execId val accumUpdates = metricsUpdate.accumUpdates + val executorMetrics = metricsUpdate.executorUpdates.map(executorMetricsToJson(_)) ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.metricsUpdate) ~ ("Executor ID" -> execId) ~ ("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) => @@ -243,7 +244,9 @@ private[spark] object JsonProtocol { ("Stage ID" -> stageId) ~ ("Stage Attempt ID" -> stageAttemptId) ~ ("Accumulator Updates" -> JArray(updates.map(accumulableInfoToJson).toList)) - }) + }) ~ + ("Executor Metrics Updated" -> executorMetrics) + } def blockUpdateToJson(blockUpdate: SparkListenerBlockUpdated): JValue = { @@ -379,6 +382,21 @@ private[spark] object JsonProtocol { ("Updated Blocks" -> updatedBlocks) } + /** + * Convert ExecutorMetrics to JSON. + * + * @param executorMetrics the executor metrics + * @return the JSON representation + */ + def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { + ("Timestamp" -> executorMetrics.timestamp) ~ + ("JVM Used Memory" -> executorMetrics.jvmUsedMemory) ~ + ("Onheap Execution Memory" -> executorMetrics.onHeapExecutionMemory) ~ + ("Offheap Execution Memory" -> executorMetrics.offHeapExecutionMemory) ~ + ("Onheap Storage Memory" -> executorMetrics.onHeapStorageMemory) ~ + ("Offheap Storage Memory" -> executorMetrics.offHeapStorageMemory) + } + def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { val reason = Utils.getFormattedClassName(taskEndReason) val json: JObject = taskEndReason match { @@ -583,6 +601,23 @@ private[spark] object JsonProtocol { SparkListenerTaskGettingResult(taskInfo) } + /** + * Extract the ExecutorMetrics from JSON. + * + * @param json the JSON representation of executor metrics + * @return the ExecutorMetrics + */ + def executorMetricsFromJson(json: JValue): ExecutorMetrics = { + val timeStamp = (json \ "Timestamp").extract[Long] + val jvmUsedMemory = (json \ "JVM Used Memory").extract[Long] + val onHeapExecutionMemory = (json \ "Onheap Execution Memory").extract[Long] + val offHeapExecutionMemory = (json \ "Offheap Execution Memory").extract[Long] + val onHeapStorageMemory = (json \ "Onheap Storage Memory").extract[Long] + val offHeapStorageMemory = (json \ "Offheap Storage Memory").extract[Long] + new ExecutorMetrics(timeStamp, jvmUsedMemory, onHeapExecutionMemory, + offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory) + } + def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { val stageId = (json \ "Stage ID").extract[Int] val stageAttemptId = @@ -689,7 +724,11 @@ private[spark] object JsonProtocol { (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) (taskId, stageId, stageAttemptId, updates) } - SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates) + val executorUpdates = jsonOption(json \ "Executor Metrics Updated") match { + case None => None + case Some(executorUpdate) => Some(executorMetricsFromJson(executorUpdate)) + } + SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates) } def blockUpdateFromJson(json: JValue): SparkListenerBlockUpdated = { 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 4fecf84db65a..9858c73fae2f 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index 4fecf84db65a..94eddfa0c7bb 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "attempts" : [ { 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 new file mode 100644 index 000000000000..1cd5fbb794cb --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json @@ -0,0 +1,367 @@ +[ { + "id" : "driver", + "hostPort" : "cluster-node0033.company.com:53121", + "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" : 1046269132, + "addTime" : "2018-03-19T18:21:24.751GMT", + "executorLogs" : { }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 1046269132, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 671982856, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 900126, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 900126, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "8", + "hostPort" : "cluster-node6128.company.com:1664", + "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" : 956615884, + "addTime" : "2018-03-19T18:22:25.988GMT", + "executorLogs" : { + "stdout" : "http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 226589176, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 0, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 0, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "7", + "hostPort" : "cluster-node2449.company.com:26914", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 11073, + "totalGCTime" : 942, + "totalInputBytes" : 36849246, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 349006, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:22:10.182GMT", + "executorLogs" : { + "stdout" : "http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 365803960, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 562630, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 562630, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "6", + "hostPort" : "cluster-node6148.company.com:1233", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 34956, + "totalGCTime" : 5484, + "totalInputBytes" : 24250210, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 242714, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:22:19.537GMT", + "executorLogs" : { + "stdout" : "http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 501950360, + "onHeapExecutionMemory" : 35651584, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 562630, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 36214214, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "5", + "hostPort" : "cluster-node5882.company.com:24590", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 30945, + "totalGCTime" : 6111, + "totalInputBytes" : 36838295, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 355051, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:22:11.267GMT", + "executorLogs" : { + "stdout" : "http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 513138800, + "onHeapExecutionMemory" : 41943040, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 562630, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 42505670, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "4", + "hostPort" : "cluster-node6137.company.com:19036", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 79365, + "totalGCTime" : 6979, + "totalInputBytes" : 50409514, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 31362123, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:21:44.950GMT", + "executorLogs" : { + "stdout" : "http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 707255176, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 83580552, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 83580552, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "3", + "hostPort" : "cluster-node1177.company.com:12614", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 2, + "totalTasks" : 2, + "totalDuration" : 38198, + "totalGCTime" : 1024, + "totalInputBytes" : 50423609, + "totalShuffleRead" : 438675, + "totalShuffleWrite" : 26424033, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:21:33.370GMT", + "executorLogs" : { + "stdout" : "http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 620936920, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 72624976, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 72624976, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "2", + "hostPort" : "cluster-node6036.company.com:17435", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 44061, + "totalGCTime" : 4677, + "totalInputBytes" : 50423423, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 22950296, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:21:41.213GMT", + "executorLogs" : { + "stdout" : "http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 695372944, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 58468552, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 58468552, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "1", + "hostPort" : "cluster-node4342.company.com:29144", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 2, + "totalTasks" : 2, + "totalDuration" : 31501, + "totalGCTime" : 936, + "totalInputBytes" : 74654808, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 20352030, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:21:33.307GMT", + "executorLogs" : { + "stdout" : "http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 505317000, + "onHeapExecutionMemory" : 35651584, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 47961406, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 83612990, + "offHeapUnifiedMemory" : 0 + } +} ] 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 79950b0dc648..bffe37ce52b4 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "attempts" : [ { @@ -28,19 +43,4 @@ "startTimeEpoch" : 1515492942372, "endTimeEpoch" : 1515493477606 } ] -}, { - "id" : "app-20161116163331-0000", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2016-11-16T22:33:29.916GMT", - "endTime" : "2016-11-16T22:33:40.587GMT", - "lastUpdated" : "", - "duration" : 10671, - "sparkUser" : "jose", - "completed" : true, - "appSparkVersion" : "2.1.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1479335609916, - "endTimeEpoch" : 1479335620587 - } ] } ] 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 7d60977dcd4f..066217dacbe9 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "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 dfbfd8aedcc2..b1226c905d6a 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "attempts" : [ { @@ -101,4 +116,4 @@ "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890 } ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 new file mode 100644 index 000000000000..26289113d369 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -0,0 +1,106 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.4.0-SNAPSHOT"} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483677960,"JVM Used Memory":587908264,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"cluster-node0033.company.com","Port":53121},"Maximum Memory":1046269132,"Timestamp":1521483684751,"Maximum Onheap Memory":1046269132,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_31/jre","Java Version":"1.8.0_31 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.jars.ivySettings":"/export/apps/spark/commonconf/ivysettings.xml","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.driver.host":"cluster-node0033.company.com","spark.dynamicAllocation.sustainedSchedulerBacklogTimeout":"5","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"36653","spark.shuffle.service.enabled":"true","spark.ui.acls.enable":"true","spark.reducer.maxSizeInFlight":"48m","spark.yarn.queue":"spark_default","spark.repl.class.uri":"spark://cluster-node0033.company.com:36653/classes","spark.jars":"","spark.yarn.historyServer.address":"clustersh01.company.com:18080","spark.memoryOverhead.multiplier.percent":"10","spark.repl.class.outputDir":"/grid/a/mapred/tmp/spark-c015ad60-cb0b-4920-a14d-e79aa91000f3/repl-c23128a3-a417-40f7-933e-3cd0a5e7c747","spark.dynamicAllocation.cachedExecutorIdleTimeout":"1200","spark.yarn.access.namenodes":"hdfs://clusternn02.company.com:9000","spark.app.name":"Spark shell","spark.dynamicAllocation.schedulerBacklogTimeout":"5","spark.yarn.security.credentials.hive.enabled":"false","spark.yarn.am.cores":"1","spark.memoryOverhead.min":"384","spark.scheduler.mode":"FIFO","spark.driver.memory":"2G","spark.executor.instances":"4","spark.isolated.classloader.additional.classes.prefix":"com_company_","spark.logConf":"true","spark.ui.showConsoleProgress":"true","spark.user.priority.jars":"*********(redacted)","spark.isolated.classloader":"true","spark.sql.sources.schemaStringLengthThreshold":"40000","spark.yarn.secondary.jars":"spark-avro_2.11-3.2.0.21.jar,grid-topology-1.0.jar","spark.reducer.maxBlocksInFlightPerAddress":"100","spark.dynamicAllocation.maxExecutors":"900","spark.yarn.appMasterEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.executor.id":"driver","spark.yarn.am.memory":"2G","spark.driver.cores":"1","spark.min.mem.vore.ratio":"5","spark.sql.sources.partitionOverwriteMode":"DYNAMIC","spark.submit.deployMode":"client","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.isolated.classloader.default.jar":"*dali-data-spark*","spark.authenticate":"true","spark.eventLog.usexattr":"true","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.memory":"2G","spark.home":"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51","spark.reducer.maxReqsInFlight":"10","spark.eventLog.dir":"hdfs://clusternn02.company.com:9000/system/spark-history","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.isolated.classes":"org.apache.hadoop.hive.ql.io.CombineHiveInputFormat$CombineHiveInputSplit","spark.eventLog.compress":"true","spark.executor.cores":"1","spark.version":"2.1.0","spark.driver.appUIAddress":"http://cluster-node0033.company.com:46527","spark.repl.local.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"clusterwp01.company.com","spark.min.memory-gb.size":"10","spark.dynamicAllocation.minExecutors":"1","spark.dynamicAllocation.initialExecutors":"3","spark.expressionencoder.org.apache.avro.specific.SpecificRecord":"com.databricks.spark.avro.AvroEncoder$","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://clusterwp01.company.com:8080/proxy/application_1506645932520_24630151","spark.executorEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.dynamicAllocation.executorIdleTimeout":"150","spark.shell.auto.node.labeling":"true","spark.yarn.dist.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.app.id":"application_1506645932520_24630151","spark.ui.view.acls":"*"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_31/jre/lib/amd64","user.dir":"*********(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.31-b07","java.endorsed.dirs":"/usr/java/jdk1.8.0_31/jre/lib/endorsed","java.runtime.version":"1.8.0_31-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_31/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_31/jre/lib/resources.jar:/usr/java/jdk1.8.0_31/jre/lib/rt.jar:/usr/java/jdk1.8.0_31/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_31/jre/lib/jsse.jar:/usr/java/jdk1.8.0_31/jre/lib/jce.jar:/usr/java/jdk1.8.0_31/jre/lib/charsets.jar:/usr/java/jdk1.8.0_31/jre/lib/jfr.jar:/usr/java/jdk1.8.0_31/jre/classes","file.encoding":"ANSI_X3.4-1968","user.timezone":"*********(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"2.6.32-504.16.2.el6.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*********(redacted)","sun.jnu.encoding":"ANSI_X3.4-1968","user.language":"*********(redacted)","java.vendor.url":"*********(redacted)","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":"*********(redacted)","user.name":"*********(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --class org.apache.spark.repl.Main --name Spark shell --jars /export/home/edlu/spark-avro_2.11-3.2.0.21.jar,/export/apps/hadoop/site/lib/grid-topology-1.0.jar --num-executors 4 spark-shell","java.home":"/usr/java/jdk1.8.0_31/jre","java.version":"1.8.0_31","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-servlet-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/derby-10.12.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-reflect-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-graphx_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-util-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/base64-2.3.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-auth-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/validation-api-1.1.0.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-api-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/objenesis-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/conf/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpclient-4.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/kryo-shaded-3.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-library-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-net-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xz-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-annotations-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/activation-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arpack_combined_all-0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libthrift-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aircompressor-0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-jackson-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/asm-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-hive_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ivy-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/export/apps/hadoop/site/etc/hadoop/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-java-1.1.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-format-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-all-4.1.17.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-ipc-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xmlenc-0.52.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jdo-api-3.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-client-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-runtime-3.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/pyrolite-4.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-catalyst_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-collections-3.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-api-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stream-2.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-format-2.3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-vector-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-web-proxy-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sketch_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-common-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hppc-0.7.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sql_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/univocity-parsers-2.5.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-math3-3.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compiler-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-1.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/java-xmlbuilder-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-annotations-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-3.9.9.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zookeeper-3.4.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-compiler-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/eigenbase-properties-1.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-yarn_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/JavaEWAH-0.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsr305-1.3.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libfb303-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.annotation-api-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-digester-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-jvm-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-framework-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/paranamer-2.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/janino-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-core-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-core-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsch-0.1.42.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-unsafe_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-codec-1.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jtransforms-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/lz4-java-1.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-core-3.2.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/core-1.1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/leveldbjni-all-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-dbcp-1.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang3-3.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill-java-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jodd-core-3.5.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-pool-1.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/minlog-1.3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/gson-2.2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/py4j-0.10.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-streaming_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/machinist_2.11-0.6.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-0.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-app-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-graphite-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-core-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib-local_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-memory-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-guava-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xercesImpl-2.9.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-tags_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javolution-5.5.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/joda-time-2.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-2.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-jobclient-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang-2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/compress-lzf-1.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-crypto-1.0.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-core-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-recipes-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guava-14.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-core_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-common_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-launcher_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr4-runtime-4.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-util-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jaxb-api-2.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-io-2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-encoding-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpcore-4.4.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-xc-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/protobuf-java-2.5.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-configuration-1.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compress-1.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-mapreduce-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ST4-4.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-core-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-shuffle-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-repl_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/opencsv-2.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-logging-1.1.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-cli-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-client-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-hdfs-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/log4j-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-column-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill_2.11-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stringtemplate-3.2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-common-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-shuffle_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-kvstore_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jta-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javassist-3.18.1-GA.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-httpclient-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jets3t-0.9.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-json-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bcprov-jdk15on-1.58.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/oro-2.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsp-api-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1506645932520_24630151","Timestamp":1521483660843,"User":"edlu"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483693307,"Executor ID":"1","Executor Info":{"Host":"cluster-node4342.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stdout?start=-4096","stderr":"http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483693370,"Executor ID":"3","Executor Info":{"Host":"cluster-node1177.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stdout?start=-4096","stderr":"http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"cluster-node4342.company.com","Port":29144},"Maximum Memory":956615884,"Timestamp":1521483693405,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"cluster-node1177.company.com","Port":12614},"Maximum Memory":956615884,"Timestamp":1521483693483,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483701213,"Executor ID":"2","Executor Info":{"Host":"cluster-node6036.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stdout?start=-4096","stderr":"http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"cluster-node6036.company.com","Port":17435},"Maximum Memory":956615884,"Timestamp":1521483701726,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483704950,"Executor ID":"4","Executor Info":{"Host":"cluster-node6137.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stdout?start=-4096","stderr":"http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"cluster-node6137.company.com","Port":19036},"Maximum Memory":956615884,"Timestamp":1521483705369,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line44.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line44.$read$$iw$$iw$$iw$$iw.(:59)\n$line44.$read$$iw$$iw$$iw.(:61)\n$line44.$read$$iw$$iw.(:63)\n$line44.$read$$iw.(:65)\n$line44.$read.(:67)\n$line44.$read$.(:71)\n$line44.$read$.()\n$line44.$eval$.$print$lzycompute(:7)\n$line44.$eval$.$print(:6)\n$line44.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1521483706188} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1521483706193} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483707958,"JVM Used Memory":511901456,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":415523,"Offheap Storage Memory":0}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line48.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line48.$read$$iw$$iw$$iw$$iw.(:59)\n$line48.$read$$iw$$iw$$iw.(:61)\n$line48.$read$$iw$$iw.(:63)\n$line48.$read$$iw.(:65)\n$line48.$read.(:67)\n$line48.$read$.(:71)\n$line48.$read$.()\n$line48.$eval$.$print$lzycompute(:7)\n$line48.$eval$.$print(:6)\n$line48.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1521483708914} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1521483708914} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483712396,"JVM Used Memory":197797104,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at :40","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- AnalysisBarrier\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nappId: string, attemptId: string, name: string, mode: string, completed: string, duration: string, endTime: string, endTimeEpoch: string, lastUpdated: string, lastUpdatedEpoch: string, sparkUser: string, startTime: string, startTimeEpoch: string, appSparkVersion: string, endDate: string, azkaban.link.workflow.url: string, azkaban.link.execution.url: string, azkaban.link.job.url: string, user.name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@babef5e, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) LocalLimit 21\n +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@babef5e, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"LocalLimit","simpleString":"LocalLimit 21","children":[{"nodeName":"Project","simpleString":"Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"InMemoryTableScan","simpleString":"InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]","children":[],"metrics":[{"name":"number of output rows","accumulatorId":35,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":36,"metricType":"timing"}]}],"metrics":[]}],"metrics":[]}],"metrics":[]}],"metrics":[{"name":"duration total (min, med, max)","accumulatorId":34,"metricType":"timing"}]}],"metrics":[]},"time":1521483710146} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1521483710385,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]}],"Stage IDs":[0,1,2],"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710427,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710514,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1521483710517,"Executor ID":"2","Host":"cluster-node6036.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"4","Host":"cluster-node6137.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483715514,"JVM Used Memory":83495872,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":36413,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483720047,"JVM Used Memory":121617408,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":36413,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483717971,"JVM Used Memory":603050152,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483722392,"JVM Used Memory":256639776,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483724389,"JVM Used Memory":431714816,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483725467,"JVM Used Memory":504156608,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":25165398,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483730046,"JVM Used Memory":292871096,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483727958,"JVM Used Memory":607069712,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1521483728324,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483728367,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"154334487","Value":"154334486","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"27714","Value":"27713","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":37809697,"Value":37809697,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":98699361,"Value":98699361,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":20002743,"Value":20002743,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":834,"Value":834,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":1856,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":13106272441,"Value":13106272441,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":15665,"Value":15665,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":741777109,"Value":741777109,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":2081,"Value":2081,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2081,"Executor Deserialize CPU Time":741777109,"Executor Run Time":15665,"Executor CPU Time":13106272441,"Result Size":1856,"JVM GC Time":834,"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":20002743,"Shuffle Write Time":98699361,"Shuffle Records Written":466636},"Input Metrics":{"Bytes Read":37809697,"Records Read":466636},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483732392,"JVM Used Memory":207788120,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":21875994,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483730182,"Executor ID":"7","Executor Info":{"Host":"cluster-node2449.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stdout?start=-4096","stderr":"http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1521483730192,"Executor ID":"7","Host":"cluster-node2449.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"cluster-node2449.company.com","Port":26914},"Maximum Memory":956615884,"Timestamp":1521483730295,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483731267,"Executor ID":"5","Executor Info":{"Host":"cluster-node5882.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stdout?start=-4096","stderr":"http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1521483731270,"Executor ID":"5","Host":"cluster-node5882.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"cluster-node5882.company.com","Port":24590},"Maximum Memory":956615884,"Timestamp":1521483731726,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483734387,"JVM Used Memory":585461008,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483735466,"JVM Used Memory":286822456,"Onheap Execution Memory":35651584,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483737958,"JVM Used Memory":619161088,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483739537,"Executor ID":"6","Executor Info":{"Host":"cluster-node6148.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stdout?start=-4096","stderr":"http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1521483739540,"Executor ID":"6","Host":"cluster-node6148.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"cluster-node6148.company.com","Port":1233},"Maximum Memory":956615884,"Timestamp":1521483739859,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483742392,"JVM Used Memory":618421104,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":72624976,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1521483730192,"Executor ID":"7","Host":"cluster-node2449.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483741265,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"1953294","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"3575","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"76","Value":"75","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"7964","Value":"7963","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"196587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"3575","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"7495","Value":"7494","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":3575,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":36849246,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":71122623,"Value":71122623,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":3575,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":349006,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":942,"Value":942,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":2437,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5608539520,"Value":5608539520,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":9646,"Value":9646,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":775466204,"Value":775466204,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1177,"Value":1177,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1177,"Executor Deserialize CPU Time":775466204,"Executor Run Time":9646,"Executor CPU Time":5608539520,"Result Size":2437,"JVM GC Time":942,"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":349006,"Shuffle Write Time":71122623,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483744387,"JVM Used Memory":533322584,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":52049755,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1521483728324,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483741988,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"63","Value":"138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"13344","Value":"21307","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"12907","Value":"20401","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":43359041,"Value":114481664,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":102,"Value":1044,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4831,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":2885767088,"Value":8494306608,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":13538,"Value":23184,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":90660797,"Value":866127001,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":104,"Value":1281,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":104,"Executor Deserialize CPU Time":90660797,"Executor Run Time":13538,"Executor CPU Time":2885767088,"Result Size":2394,"JVM GC Time":102,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":43359041,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483745464,"JVM Used Memory":504693976,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483746631,"JVM Used Memory":365802552,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483745988,"Executor ID":"8","Executor Info":{"Host":"cluster-node6128.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stdout?start=-4096","stderr":"http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"8","Host":"cluster-node6128.company.com","Port":1664},"Maximum Memory":956615884,"Timestamp":1521483746284,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483746534,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"206421303","Value":"360755789","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"65190","Value":"92903","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423609,"Value":88233306,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":123692332,"Value":222391693,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":26424033,"Value":46426776,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":878,"Value":1712,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":3712,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":19065776829,"Value":32172049270,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":35010,"Value":50675,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":732371255,"Value":1474148364,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":942,"Value":3023,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":942,"Executor Deserialize CPU Time":732371255,"Executor Run Time":35010,"Executor CPU Time":19065776829,"Result Size":1856,"JVM GC Time":878,"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":26424033,"Shuffle Write Time":123692332,"Shuffle Records Written":624246},"Input Metrics":{"Bytes Read":50423609,"Records Read":624246},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483750045,"JVM Used Memory":207536536,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":2461716,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483747958,"JVM Used Memory":641729640,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"5","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483751931,"JVM Used Memory":513138800,"Onheap Execution Memory":65536,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483752392,"JVM Used Memory":619676000,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483754387,"JVM Used Memory":695372944,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468552,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483755464,"JVM Used Memory":504694360,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483756630,"JVM Used Memory":365802904,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1521483710517,"Executor ID":"2","Host":"cluster-node6036.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483754578,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"73812","Value":"166715","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":214358240,"Value":436749933,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":3,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":4677,"Value":6389,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":24023445128,"Value":56195494398,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":42072,"Value":92747,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":952244001,"Value":2426392365,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":1796,"Value":4819,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1796,"Executor Deserialize CPU Time":952244001,"Executor Run Time":42072,"Executor CPU Time":24023445128,"Result Size":1856,"JVM GC Time":4677,"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":22950296,"Shuffle Write Time":214358240,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"6","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483758435,"JVM Used Memory":397749936,"Onheap Execution Memory":65536,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483760046,"JVM Used Memory":488354552,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":25910376,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483757958,"JVM Used Memory":662984224,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483762392,"JVM Used Memory":619676384,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"5","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483762915,"JVM Used Memory":353487552,"Onheap Execution Memory":41943040,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1521483731270,"Executor ID":"5","Host":"cluster-node5882.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483762215,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"5784372","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"10678","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"125829116","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"131","Value":"269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"22823","Value":"44130","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"586742","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"10679","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"19841","Value":"40242","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":10679,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":110532652,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":215191844,"Value":329673508,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":10678,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1053344,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":125829120,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":9,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":6111,"Value":7155,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":7268,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":8017651595,"Value":16511958203,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":27779,"Value":50963,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1058013355,"Value":1924140356,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":2357,"Value":3638,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2357,"Executor Deserialize CPU Time":1058013355,"Executor Run Time":27779,"Executor CPU Time":8017651595,"Result Size":2437,"JVM GC Time":6111,"Result Serialization Time":9,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355051,"Shuffle Write Time":215191844,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483765464,"JVM Used Memory":504694712,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483766630,"JVM Used Memory":365803256,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483767180,"JVM Used Memory":226589112,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"6","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483768433,"JVM Used Memory":501950360,"Onheap Execution Memory":35651584,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483770047,"JVM Used Memory":180202344,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":55709919,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483767958,"JVM Used Memory":665764240,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483772392,"JVM Used Memory":619676736,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483775464,"JVM Used Memory":504695064,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483776630,"JVM Used Memory":365803608,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1521483739540,"Executor ID":"6","Host":"cluster-node6148.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483774496,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"300","Value":"569","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"26291","Value":"70421","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"24092","Value":"64334","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":521474789,"Value":851148297,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":13,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":5484,"Value":12639,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9705,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":7239844597,"Value":23751802800,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":30603,"Value":81566,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1062050912,"Value":2986191268,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3790,"Value":7428,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3790,"Executor Deserialize CPU Time":1062050912,"Executor Run Time":30603,"Executor CPU Time":7239844597,"Result Size":2437,"JVM GC Time":5484,"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":242714,"Shuffle Write Time":521474789,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710514,"Completion Time":1521483774499,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":7428,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":23751802800,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":81566,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":851148297,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":12639,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2986191268,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"70421","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9705,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"569","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"64334","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483777175,"JVM Used Memory":226589144,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483780046,"JVM Used Memory":639104112,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":83580552,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483777958,"JVM Used Memory":669979432,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483782392,"JVM Used Memory":620936568,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483785464,"JVM Used Memory":505317000,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483786630,"JVM Used Memory":365803960,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483787175,"JVM Used Memory":226589176,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483790047,"JVM Used Memory":707255176,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69534656,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483787958,"JVM Used Memory":671982856,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483792392,"JVM Used Memory":620936920,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"4","Host":"cluster-node6137.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483789895,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"131866","Value":"298581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":464179545,"Value":900929478,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":6979,"Value":13368,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":25321337014,"Value":81516831412,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":74896,"Value":167643,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1173542343,"Value":3599934708,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":4315,"Value":9134,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4315,"Executor Deserialize CPU Time":1173542343,"Executor Run Time":74896,"Executor CPU Time":25321337014,"Result Size":1856,"JVM GC Time":6979,"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":31362123,"Shuffle Write Time":464179545,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710427,"Completion Time":1521483789896,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3599934708,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"298581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":81516831412,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":9134,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":900929478,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":167643,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":13368,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483789907,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1521483789924,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483794387,"JVM Used Memory":317344968,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468552,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1521483789924,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1521483792118,"Failed":false,"Killed":false,"Accumulables":[{"ID":34,"Name":"duration total (min, med, max)","Update":"1","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":35,"Name":"number of output rows","Update":"6928","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Update":"1181","Value":"1180","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"number of output rows","Update":"10945","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"number of output rows","Update":"62","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Update":"33619967","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"sort time total (min, med, max)","Update":"6","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":22,"Name":"duration total (min, med, max)","Update":"896","Value":"895","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"peak memory total (min, med, max)","Update":"34078719","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Update":"53","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"duration total (min, med, max)","Update":"1109","Value":"1108","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Update":11007,"Value":11007,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":122827,"Value":122827,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":315848,"Value":315848,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":7,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Update":67698688,"Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":92,"Name":"internal.metrics.jvmGCTime","Update":146,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Update":4685,"Value":4685,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Update":970272606,"Value":970272606,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.executorRunTime","Update":1923,"Value":1923,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Update":162560302,"Value":162560302,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Update":230,"Value":230,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":230,"Executor Deserialize CPU Time":162560302,"Executor Run Time":1923,"Executor CPU Time":970272606,"Result Size":4685,"JVM GC Time":146,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":7,"Local Blocks Fetched":1,"Fetch Wait Time":1,"Remote Bytes Read":315848,"Remote Bytes Read To Disk":0,"Local Bytes Read":122827,"Total Records Read":11007},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":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":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483789907,"Completion Time":1521483792119,"Accumulables":[{"ID":92,"Name":"internal.metrics.jvmGCTime","Value":146,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Value":11007,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"number of output rows","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":89,"Name":"internal.metrics.executorRunTime","Value":1923,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":7,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"number of output rows","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":315848,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Value":4685,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"peak memory total (min, med, max)","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"duration total (min, med, max)","Value":"895","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Value":162560302,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"duration total (min, med, max)","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Value":"1180","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"sort time total (min, med, max)","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Value":230,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Value":970272606,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"number of output rows","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":122827,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"duration total (min, med, max)","Value":"1108","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1521483792127,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1521483792153} +{"Event":"SparkListenerUnpersistRDD","RDD ID":2} +{"Event":"SparkListenerUnpersistRDD","RDD ID":20} +{"Event":"SparkListenerApplicationEnd","Timestamp":1521483792501} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 88916488c0de..4f8081f2e942 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -29,7 +29,7 @@ import org.mockito.Matchers._ import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -77,7 +77,7 @@ class HeartbeatReceiverSuite heartbeatReceiverClock = new ManualClock heartbeatReceiver = new HeartbeatReceiver(sc, heartbeatReceiverClock) heartbeatReceiverRef = sc.env.rpcEnv.setupEndpoint("heartbeat", heartbeatReceiver) - when(scheduler.executorHeartbeatReceived(any(), any(), any())).thenReturn(true) + when(scheduler.executorHeartbeatReceived(any(), any(), any(), any())).thenReturn(true) } /** @@ -213,8 +213,9 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) + val executorUpdates = new ExecutorMetrics(1234567L, 123456L, 12345L, 1234L, 123L, 12L) val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( - Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId)) + Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId, executorUpdates)) if (executorShouldReregister) { assert(response.reregisterBlockManager) } else { @@ -223,7 +224,8 @@ class HeartbeatReceiverSuite verify(scheduler).executorHeartbeatReceived( Matchers.eq(executorId), Matchers.eq(Array(1L -> metrics.accumulators())), - Matchers.eq(blockManagerId)) + Matchers.eq(blockManagerId), + Matchers.eq(executorUpdates)) } } 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 87f12f303cd5..635dae6dd9a3 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 @@ -127,6 +127,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "succeeded&failed job list json" -> "applications/local-1422981780767/jobs?status=succeeded&status=failed", "executor list json" -> "applications/local-1422981780767/executors", + "executor list with executor metrics json" -> + "applications/application_1506645932520_24630151/executors", "stage list json" -> "applications/local-1422981780767/stages", "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 8b6ec37625ee..9a06c83419c5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -30,6 +30,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} @@ -120,7 +121,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean = true + blockManagerId: BlockManagerId, + executorUpdates: ExecutorMetrics): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -634,7 +636,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean = true + blockManagerId: BlockManagerId, + executorMetrics: ExecutorMetrics): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None 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 a9e92fa07b9d..59645b2cf181 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.scheduler import java.io.{File, FileOutputStream, InputStream, IOException} +import scala.collection.immutable.Map import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import scala.io.Source import org.apache.hadoop.fs.Path @@ -29,11 +31,14 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} + /** * Test whether EventLoggingListener logs events properly. * @@ -137,6 +142,10 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit "a fine:mind$dollar{bills}.1", None, Some("lz4"))) } + test("Executor metrics update") { + testExecutorMetricsUpdateEventLogging() + } + /* ----------------- * * Actual test logic * * ----------------- */ @@ -251,6 +260,163 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } + /** + * Test executor metrics update logging functionality. This checks that a + * SparkListenerExecutorMetricsUpdate event is added to the Spark history + * log if one of the executor metrics is larger than any previously + * recorded value for the metric, per executor per stage. The task metrics + * should not be added. + */ + private def testExecutorMetricsUpdateEventLogging() { + val conf = getLoggingConf(testDirPath, None) + val logName = "executorMetricsUpdated-test" + val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) + val listenerBus = new LiveListenerBus(conf) + + // list of events and if they should be logged + val events = Array( + (SparkListenerApplicationStart("executionMetrics", None, + 1L, "update", None), true), + (createExecutorAddedEvent(1), true), + (createExecutorAddedEvent(2), true), + (createStageSubmittedEvent(0), true), + (createExecutorMetricsUpdateEvent(1, 10L, 5000L, 50L, 0L, 0L, 0L), true), // new stage + (createExecutorMetricsUpdateEvent(2, 10L, 3500L, 20L, 0L, 0L, 0L), true), // new stage + (createExecutorMetricsUpdateEvent(1, 15L, 4000L, 50L, 0L, 0L, 0L), false), + (createExecutorMetricsUpdateEvent(2, 15L, 3500L, 10L, 0L, 20L, 0L), true), // onheap storage + (createExecutorMetricsUpdateEvent(1, 20L, 6000L, 50L, 0L, 30L, 0L), true), // JVM used + (createExecutorMetricsUpdateEvent(2, 20L, 3500L, 15L, 0L, 20L, 0L), true), // onheap unified + (createStageSubmittedEvent(1), true), + (createExecutorMetricsUpdateEvent(1, 25L, 3000L, 15L, 0L, 0L, 0L), true), // new stage + (createExecutorMetricsUpdateEvent(2, 25L, 6000L, 50L, 0L, 0L, 0L), true), // new stage + (createStageCompletedEvent(0), true), + (createExecutorMetricsUpdateEvent(1, 30L, 3000L, 20L, 0L, 0L, 0L), true), // onheap execution + (createExecutorMetricsUpdateEvent(2, 30L, 5500L, 20L, 0L, 0L, 0L), false), + (createExecutorMetricsUpdateEvent(1, 35L, 3000L, 5L, 25L, 0L, 0L), true), // offheap execution + (createExecutorMetricsUpdateEvent(2, 35L, 5500L, 25L, 0L, 0L, 30L), true), // offheap storage + (createExecutorMetricsUpdateEvent(1, 40L, 3000L, 8L, 20L, 0L, 0L), false), + (createExecutorMetricsUpdateEvent(2, 40L, 5500L, 25L, 0L, 0L, 30L), false), + (createStageCompletedEvent(1), true), + (SparkListenerApplicationEnd(1000L), true)) + + // play the events for the event logger + eventLogger.start() + listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem])) + listenerBus.addToEventLogQueue(eventLogger) + for ((event, included) <- events) { + listenerBus.post(event) + } + listenerBus.stop() + eventLogger.stop() + + // Verify the log file contains the expected events + val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + try { + val lines = readLines(logData) + val logStart = SparkListenerLogStart(SPARK_VERSION) + assert(lines.size === 19) + assert(lines(0).contains("SparkListenerLogStart")) + assert(lines(1).contains("SparkListenerApplicationStart")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) + var i = 1 + for ((event, included) <- events) { + if (included) { + checkEvent(lines(i), event) + i += 1 + } + } + } finally { + logData.close() + } + } + + /** Create a stage submitted event for the specified stage Id. */ + private def createStageSubmittedEvent(stageId: Int) = + SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + + /** Create a stage completed event for the specified stage Id. */ + private def createStageCompletedEvent(stageId: Int) = + SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + + /** 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)) + + /** Create an executor metrics update event, with the specified executor metrics values. */ + private def createExecutorMetricsUpdateEvent( + executorId: Int, time: Long, + jvmUsedMemory: Long, + onHeapExecutionMemory: Long, + offHeapExecutionMemory: Long, + onHeapStorageMemory: Long, + offHeapStorageMemory: Long): SparkListenerExecutorMetricsUpdate = { + val taskMetrics = TaskMetrics.empty + taskMetrics.incDiskBytesSpilled(111) + taskMetrics.incMemoryBytesSpilled(222) + val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) + val executorUpdates = new ExecutorMetrics(time, jvmUsedMemory, onHeapExecutionMemory, + offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory) + SparkListenerExecutorMetricsUpdate( executorId.toString, accum, Some(executorUpdates)) + } + + /** Check that the two ExecutorMetrics match */ + private def checkExecutorMetrics( + executorMetrics1: Option[ExecutorMetrics], + executorMetrics2: Option[ExecutorMetrics]) = { + executorMetrics1 match { + case Some(e1) => + executorMetrics2 match { + case Some(e2) => + assert(e1.timestamp === e2.timestamp) + assert(e1.jvmUsedMemory === e2.jvmUsedMemory) + assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory) + assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory) + assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory) + assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory) + } + case None => + assert(false) + case None => + assert(executorMetrics2.isEmpty) + } + } + + /** Check that the Spark history log line matches the expected event. */ + private def checkEvent(line: String, event: SparkListenerEvent): Unit = { + assert(line.contains(event.getClass.toString.split("\\.").last)) + event match { + case executorMetrics: SparkListenerExecutorMetricsUpdate => + JsonProtocol.sparkEventFromJson(parse(line)) match { + case executorMetrics2: SparkListenerExecutorMetricsUpdate => + assert(executorMetrics.execId === executorMetrics2.execId) + assert(executorMetrics2.accumUpdates.isEmpty) + checkExecutorMetrics(executorMetrics.executorUpdates, executorMetrics2.executorUpdates) + case _ => + assertTypeError("expecting SparkListenerExecutorMetricsUpdate") + } + case stageSubmitted: SparkListenerStageSubmitted => + // accumulables can be different, so only check the stage Id + JsonProtocol.sparkEventFromJson(parse(line)) match { + case logStageSubmitted : SparkListenerStageSubmitted => + assert(logStageSubmitted.stageInfo.stageId == stageSubmitted.stageInfo.stageId) + case _ => + assertTypeError("expecting SparkListenerStageSubmitted") + } + case stageCompleted: SparkListenerStageCompleted => + // accumulables can be different, so only check the stage Id + JsonProtocol.sparkEventFromJson(parse(line)) match { + case logStageSubmitted : SparkListenerStageSubmitted => + assert(logStageSubmitted.stageInfo.stageId == stageCompleted.stageInfo.stageId) + case _ => + assertTypeError("expecting SparkListenerStageCompleted") + } + case _ => + assert(JsonProtocol.sparkEventFromJson(parse(line)) === event) + } + } + private def readLines(in: InputStream): Seq[String] = { Source.fromInputStream(in).getLines().toSeq } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index a4e4ea7cd289..149e58fc5a90 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -89,5 +90,6 @@ private class DummyTaskScheduler extends TaskScheduler { def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean = true + blockManagerId: BlockManagerId, + executorMetrics: ExecutorMetrics): Boolean = true } 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 74b72d940eee..e0cc3c46017c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -94,7 +94,8 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates))) + val executorUpdates = Some(new ExecutorMetrics(1234567L, 123456L, 12345L, 1234L, 123L, 12L)) + SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) } val blockUpdated = SparkListenerBlockUpdated(BlockUpdatedInfo(BlockManagerId("Stars", @@ -124,6 +125,7 @@ class JsonProtocolSuite extends SparkFunSuite { testEvent(nodeUnblacklisted, nodeUnblacklistedJsonString) testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) testEvent(blockUpdated, blockUpdatedJsonString) + testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) } test("Dependent Classes") { @@ -419,6 +421,30 @@ class JsonProtocolSuite extends SparkFunSuite { exceptionFailure.accumUpdates, oldExceptionFailure.accumUpdates, (x, y) => x == y) } + test("ExecutorMetricsUpdate backward compatibility: executor metrics update") { + // executorMetricsUpdate was added in 2.1.0. For older event logs, this should + // be set to None. + val executorMetricsUpdate = makeExecutorMetricsUpdate("1", true, true) + val oldExecutorMetricsUpdateJson = + JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) + .removeField( _._1 == "Executor Metrics Updated") + val exepectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", true, false) + assertEquals(exepectedExecutorMetricsUpdate, + JsonProtocol.executorMetricsUpdateFromJson(oldExecutorMetricsUpdateJson)) + } + + test("ExecutorMetricsUpdate: empty metrics update") { + // For SparkListenerExecutorMetricUpdate events, metrics update will be set to + // empty in the event log, to avoid excess logging. + val executorMetricsUpdate = makeExecutorMetricsUpdate("1", true, true) + val oldExecutorMetricsUpdateJson = + JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) + .replace(List("Metrics Updated"), List.empty[JValue]) + val exepectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", false, true) + assertEquals(exepectedExecutorMetricsUpdate, + JsonProtocol.executorMetricsUpdateFromJson(oldExecutorMetricsUpdateJson)) + } + test("AccumulableInfo value de/serialization") { import InternalAccumulator._ val blocks = Seq[(BlockId, BlockStatus)]( @@ -565,6 +591,7 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(stageAttemptId1 === stageAttemptId2) assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) }) + assertEquals(e1.executorUpdates, e2.executorUpdates) case (e1, e2) => assert(e1 === e2) case _ => fail("Events don't match in types!") @@ -654,6 +681,25 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(metrics1.bytesRead === metrics2.bytesRead) } + private def assertEquals(metrics1: Option[ExecutorMetrics], metrics2: Option[ExecutorMetrics]) { + metrics1 match { + case Some(m1) => + metrics2 match { + case Some(m2) => + assert(m1.timestamp === m2.timestamp) + assert(m1.jvmUsedMemory === m2.jvmUsedMemory) + assert(m1.onHeapExecutionMemory === m2.onHeapExecutionMemory) + assert(m1.offHeapExecutionMemory === m2.offHeapExecutionMemory) + assert(m1.onHeapStorageMemory === m2.onHeapStorageMemory) + assert(m1.offHeapStorageMemory === m2.offHeapStorageMemory) + case None => + assert(false) + } + case None => + assert(metrics2.isEmpty) + } + } + private def assertEquals(result1: JobResult, result2: JobResult) { (result1, result2) match { case (JobSucceeded, JobSucceeded) => @@ -820,6 +866,26 @@ private[spark] object JsonProtocolSuite extends Assertions { new AccumulableInfo(id, Some(s"Accumulable$id"), Some(s"delta$id"), Some(s"val$id"), internal, countFailedValues, metadata) + /** Creates an SparkListenerExecutorMetricsUpdate event */ + private def makeExecutorMetricsUpdate(execId: String, + includeTaskMetrics: Boolean, + includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { + val taskMetrics = + if (includeTaskMetrics) { + Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), + makeAccumulableInfo(2, false, false, None)))) + } else { + Seq() + } + val executorMetricsUpdate = + if (includeExecutorMetrics) { + Some(new ExecutorMetrics(1234567L, 123456L, 0L, 0L, 0L, 0L)) + } else { + None + } + SparkListenerExecutorMetricsUpdate(execId, taskMetrics, executorMetricsUpdate) + } + /** * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is * set to true) or read data from a shuffle otherwise. @@ -2007,7 +2073,16 @@ private[spark] object JsonProtocolSuite extends Assertions { | } | ] | } - | ] + | ], + | "Executor Metrics Updated" : { + | "Timestamp" : 1234567, + | "JVM Used Memory" : 123456, + | "Onheap Execution Memory" : 12345, + | "Offheap Execution Memory" : 1234, + | "Onheap Storage Memory" : 123, + | "Offheap Storage Memory" : 12 + | } + | |} """.stripMargin diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 9552d001a079..698921456109 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -77,6 +77,7 @@ app-20180109111548-0000 app-20161115172038-0000 app-20161116163331-0000 application_1516285256255_0012 +application_1506645932520_24630151 local-1422981759269 local-1422981780767 local-1425081759269 From 5d6ae1c34bf6618754e4b8b2e756a9a7b4bad987 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Sun, 1 Apr 2018 19:13:41 -0700 Subject: [PATCH 02/29] modify MimaExcludes.scala to filter changes to SparkListenerExecutorMetricsUpdate --- project/MimaExcludes.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7d0e88ee20c3..8ee7822282e4 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -74,6 +74,12 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.Node"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.this"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.this") + + // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate$") ) // Exclude rules for 2.3.x From ad10d2814bbfbaf8c21fcbb1abe83ef7a8e9ffe7 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Sat, 21 Apr 2018 17:02:57 -0700 Subject: [PATCH 03/29] Address code review comments, change event logging to stage end. --- .../org/apache/spark/executor/Executor.scala | 56 +++- .../spark/executor/ExecutorMetrics.scala | 19 +- .../apache/spark/scheduler/DAGScheduler.scala | 16 +- .../scheduler/EventLoggingListener.scala | 47 ++-- .../spark/scheduler/PeakExecutorMetrics.scala | 105 ++++--- .../org/apache/spark/status/LiveEntity.scala | 2 +- .../org/apache/spark/status/api/v1/api.scala | 7 +- .../org/apache/spark/util/JsonProtocol.scala | 22 +- .../application_list_json_expectation.json | 10 +- .../completed_app_list_json_expectation.json | 10 +- ...ith_executor_metrics_json_expectation.json | 259 +++++++----------- .../limit_app_list_json_expectation.json | 10 +- .../minDate_app_list_json_expectation.json | 10 +- .../minEndDate_app_list_json_expectation.json | 10 +- .../application_1506645932520_24630151 | 163 ++++------- .../apache/spark/HeartbeatReceiverSuite.scala | 3 +- .../scheduler/EventLoggingListenerSuite.scala | 189 +++++++++---- .../spark/status/AppStatusListenerSuite.scala | 106 ++++++- .../apache/spark/util/JsonProtocolSuite.scala | 48 ++-- project/MimaExcludes.scala | 2 +- 20 files changed, 642 insertions(+), 452 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 16a7c58dade7..bbc5f375bc9f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -19,12 +19,13 @@ package org.apache.spark.executor import java.io.{File, NotSerializableException} import java.lang.Thread.UncaughtExceptionHandler -import java.lang.management.ManagementFactory +import java.lang.management.{BufferPoolMXBean, ManagementFactory} import java.net.{URI, URL} import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent._ import javax.annotation.concurrent.GuardedBy +import javax.management.ObjectName import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -36,7 +37,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} +import org.apache.spark.memory.{MemoryManager, SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription} import org.apache.spark.shuffle.FetchFailedException @@ -71,6 +72,12 @@ private[spark] class Executor( private val conf = env.conf + // BufferPoolMXBean for direct memory + private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME) + + // BufferPoolMXBean for mapped memory + private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME) + // No ip or host:port - just hostname Utils.checkHost(executorHostname) // must not have port specified. @@ -773,10 +780,8 @@ private[spark] class Executor( val curGCTime = computeTotalGcTime() // get executor level memory metrics - val executorUpdates = new ExecutorMetrics(System.currentTimeMillis(), - ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), - env.memoryManager.onHeapExecutionMemoryUsed, env.memoryManager.offHeapExecutionMemoryUsed, - env.memoryManager.onHeapStorageMemoryUsed, env.memoryManager.offHeapStorageMemoryUsed) + val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager, + directBufferPool, mappedBufferPool) for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { @@ -814,4 +819,43 @@ private[spark] object Executor { // task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be // used instead. val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties] + + val DIRECT_BUFFER_POOL_NAME = "direct" + val MAPPED_BUFFER_POOL_NAME = "mapped" + + /** Get the BufferPoolMXBean for the specified buffer pool. */ + def getBufferPool(pool: String): BufferPoolMXBean = { + val name = new ObjectName("java.nio:type=BufferPool,name=" + pool) + ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, + name.toString, classOf[BufferPoolMXBean]) + } + + /** + * Get the current executor level memory metrics. + * + * @param memoryManager the memory manager + * @param direct the direct memory buffer pool + * @param mapped the mapped memory buffer pool + * @return the executor memory metrics + */ + def getCurrentExecutorMetrics( + memoryManager: MemoryManager, + direct: BufferPoolMXBean, + mapped: BufferPoolMXBean) : ExecutorMetrics = { + val onHeapExecutionMemoryUsed = memoryManager.onHeapExecutionMemoryUsed + val offHeapExecutionMemoryUsed = memoryManager.offHeapExecutionMemoryUsed + val onHeapStorageMemoryUsed = memoryManager.onHeapStorageMemoryUsed + val offHeapStorageMemoryUsed = memoryManager.offHeapStorageMemoryUsed + new ExecutorMetrics(System.currentTimeMillis(), + ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), + ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed(), + onHeapExecutionMemoryUsed, + offHeapExecutionMemoryUsed, + onHeapStorageMemoryUsed, + offHeapStorageMemoryUsed, + onHeapExecutionMemoryUsed + onHeapStorageMemoryUsed, // on heap unified memory + offHeapExecutionMemoryUsed + offHeapStorageMemoryUsed, // off heap unified memory + direct.getMemoryUsed, + mapped.getMemoryUsed) + } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 906bdc41d05b..6d35a1c682e2 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -26,18 +26,29 @@ import org.apache.spark.annotation.DeveloperApi * This is sent to the driver periodically (on executor heartbeat), to provide * information about each executor's metrics. * - * @param timestamp the time the metrics were collected - * @param jvmUsedMemory the amount of JVM used memory for the executor + * @param timestamp the time the metrics were collected, or -1 for Spark history + * log events which are logged when a stage has completed + * @param jvmUsedHeapMemory the amount of JVM used heap memory for the executor + * @param jvmUsedNonHeapMemory the amount of JVM used non-heap memory for the executor * @param onHeapExecutionMemory the amount of on heap execution memory used * @param offHeapExecutionMemory the amount of off heap execution memory used * @param onHeapStorageMemory the amount of on heap storage memory used * @param offHeapStorageMemory the amount of off heap storage memory used + * @param onHeapUnifiedMemory the amount of on heap unified region memory used + * @param offHeapUnifiedMemory the amount of off heap unified region memory used + * @param directMemory the amount of direct memory used + * @param mappedMemory the amount of mapped memory used */ @DeveloperApi class ExecutorMetrics private[spark] ( val timestamp: Long, - val jvmUsedMemory: Long, + val jvmUsedHeapMemory: Long, + val jvmUsedNonHeapMemory: Long, val onHeapExecutionMemory: Long, val offHeapExecutionMemory: Long, val onHeapStorageMemory: Long, - val offHeapStorageMemory: Long) extends Serializable + val offHeapStorageMemory: Long, + val onHeapUnifiedMemory: Long, + val offHeapUnifiedMemory: Long, + val directMemory: Long, + val mappedMemory: Long) extends Serializable diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 6ad7eaae162c..5b628929ed3d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -35,7 +35,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.{Executor, ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils @@ -214,6 +214,12 @@ class DAGScheduler( private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + /** BufferPoolMXBean for direct memory */ + private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME) + + /** BufferPoolMXBean for mapped memory */ + private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME) + /** * Called by the TaskSetManager to report task's starting. */ @@ -1766,12 +1772,8 @@ class DAGScheduler( /** Reports heartbeat metrics for the driver. */ private def reportHeartBeat(): Unit = { // get driver memory metrics - val driverUpdates = new ExecutorMetrics(System.currentTimeMillis(), - ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), - sc.env.memoryManager.onHeapExecutionMemoryUsed, - sc.env.memoryManager.offHeapExecutionMemoryUsed, - sc.env.memoryManager.onHeapStorageMemoryUsed, - sc.env.memoryManager.offHeapStorageMemoryUsed) + val driverUpdates = Executor.getCurrentExecutorMetrics( + sc.env.memoryManager, directBufferPool, mappedBufferPool) val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, Some(driverUpdates))) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index e159421f4fbb..5fae14bb1708 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -94,8 +94,9 @@ private[spark] class EventLoggingListener( // Visible for tests only. private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) - // Peak metric values for each executor - private var peakExecutorMetrics = new mutable.HashMap[String, PeakExecutorMetrics]() + // map of live stages, to peak executor metrics for the stage + private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int), + mutable.HashMap[String, PeakExecutorMetrics]]() /** * Creates the log file in the configured log directory. @@ -162,7 +163,8 @@ private[spark] class EventLoggingListener( override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { logEvent(event) // clear the peak metrics when a new stage starts - peakExecutorMetrics.values.foreach(_.reset()) + liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), + new mutable.HashMap[String, PeakExecutorMetrics]()) } override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) @@ -177,6 +179,27 @@ private[spark] class EventLoggingListener( // Events that trigger a flush override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { + // log the peak executor metrics for the stage, for each executor + val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() + val executorMap = liveStageExecutorMetrics.remove( + (event.stageInfo.stageId, event.stageInfo.attemptNumber())) + executorMap.foreach { + executorEntry => { + for ((executorId, peakExecutorMetrics) <- executorEntry) { + val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory, + peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory, + peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory, + peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory, + peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory, + peakExecutorMetrics.mappedMemory) + val executorUpdate = new SparkListenerExecutorMetricsUpdate( + executorId, accumUpdates, Some(executorMetrics)) + logEvent(executorUpdate) + } + } + } + + // log stage completed event logEvent(event, flushLogger = true) } @@ -205,12 +228,10 @@ private[spark] class EventLoggingListener( } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { logEvent(event, flushLogger = true) - peakExecutorMetrics.put(event.executorId, new PeakExecutorMetrics()) } override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { logEvent(event, flushLogger = true) - peakExecutorMetrics.remove(event.executorId) } override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { @@ -244,19 +265,13 @@ private[spark] class EventLoggingListener( } } - /** - * Log if there is a new peak value for one of the memory metrics for the given executor. - * Metrics are cleared out when a new stage is started in onStageSubmitted, so this will - * log new peak memory metric values per executor per stage. - */ override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - var log: Boolean = false + // For the active stages, record any new peak values for the memory metrics for the executor event.executorUpdates.foreach { executorUpdates => - val peakMetrics = peakExecutorMetrics.getOrElseUpdate(event.execId, new PeakExecutorMetrics()) - if (peakMetrics.compareAndUpdate(executorUpdates)) { - val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() - logEvent(new SparkListenerExecutorMetricsUpdate(event.execId, accumUpdates, - event.executorUpdates), flushLogger = true) + liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => + val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + event.execId, new PeakExecutorMetrics()) + peakMetrics.compareAndUpdate(executorUpdates) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index 187cc77751e6..d554c938f5e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -21,17 +21,40 @@ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.status.api.v1.PeakMemoryMetrics /** - * Records the peak values for executor level metrics. If jvmUsedMemory is -1, then no values have - * been recorded yet. + * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no + * values have been recorded yet. */ private[spark] class PeakExecutorMetrics { - private var jvmUsedMemory = -1L; - private var onHeapExecutionMemory = 0L - private var offHeapExecutionMemory = 0L - private var onHeapStorageMemory = 0L - private var offHeapStorageMemory = 0L - private var onHeapUnifiedMemory = 0L - private var offHeapUnifiedMemory = 0L + private var _jvmUsedHeapMemory = -1L; + private var _jvmUsedNonHeapMemory = 0L; + private var _onHeapExecutionMemory = 0L + private var _offHeapExecutionMemory = 0L + private var _onHeapStorageMemory = 0L + private var _offHeapStorageMemory = 0L + private var _onHeapUnifiedMemory = 0L + private var _offHeapUnifiedMemory = 0L + private var _directMemory = 0L + private var _mappedMemory = 0L + + def jvmUsedHeapMemory: Long = _jvmUsedHeapMemory + + def jvmUsedNonHeapMemory: Long = _jvmUsedNonHeapMemory + + def onHeapExecutionMemory: Long = _onHeapExecutionMemory + + def offHeapExecutionMemory: Long = _offHeapExecutionMemory + + def onHeapStorageMemory: Long = _onHeapStorageMemory + + def offHeapStorageMemory: Long = _offHeapStorageMemory + + def onHeapUnifiedMemory: Long = _onHeapUnifiedMemory + + def offHeapUnifiedMemory: Long = _offHeapUnifiedMemory + + def directMemory: Long = _directMemory + + def mappedMemory: Long = _mappedMemory /** * Compare the specified memory values with the saved peak executor memory @@ -43,36 +66,44 @@ private[spark] class PeakExecutorMetrics { def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { var updated: Boolean = false - if (executorMetrics.jvmUsedMemory > jvmUsedMemory) { - jvmUsedMemory = executorMetrics.jvmUsedMemory + if (executorMetrics.jvmUsedHeapMemory > _jvmUsedHeapMemory) { + _jvmUsedHeapMemory = executorMetrics.jvmUsedHeapMemory + updated = true + } + if (executorMetrics.jvmUsedNonHeapMemory > _jvmUsedNonHeapMemory) { + _jvmUsedNonHeapMemory = executorMetrics.jvmUsedNonHeapMemory + updated = true + } + if (executorMetrics.onHeapExecutionMemory > _onHeapExecutionMemory) { + _onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory updated = true } - if (executorMetrics.onHeapExecutionMemory > onHeapExecutionMemory) { - onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory + if (executorMetrics.offHeapExecutionMemory > _offHeapExecutionMemory) { + _offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory updated = true } - if (executorMetrics.offHeapExecutionMemory > offHeapExecutionMemory) { - offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory + if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) { + _onHeapStorageMemory = executorMetrics.onHeapStorageMemory updated = true } - if (executorMetrics.onHeapStorageMemory > onHeapStorageMemory) { - onHeapStorageMemory = executorMetrics.onHeapStorageMemory + if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) { + _offHeapStorageMemory = executorMetrics.offHeapStorageMemory updated = true } - if (executorMetrics.offHeapStorageMemory > offHeapStorageMemory) { - offHeapStorageMemory = executorMetrics.offHeapStorageMemory + if (executorMetrics.onHeapUnifiedMemory > _onHeapUnifiedMemory) { + _onHeapUnifiedMemory = executorMetrics.onHeapUnifiedMemory updated = true } - val newOnHeapUnifiedMemory = (executorMetrics.onHeapExecutionMemory + - executorMetrics.onHeapStorageMemory) - if (newOnHeapUnifiedMemory > onHeapUnifiedMemory) { - onHeapUnifiedMemory = newOnHeapUnifiedMemory + if (executorMetrics.offHeapUnifiedMemory > _offHeapUnifiedMemory) { + _offHeapUnifiedMemory = executorMetrics.offHeapUnifiedMemory updated = true } - val newOffHeapUnifiedMemory = (executorMetrics.offHeapExecutionMemory + - executorMetrics.offHeapStorageMemory) - if ( newOffHeapUnifiedMemory > offHeapUnifiedMemory) { - offHeapUnifiedMemory = newOffHeapUnifiedMemory + if (executorMetrics.directMemory > _directMemory) { + _directMemory = executorMetrics.directMemory + updated = true + } + if (executorMetrics.mappedMemory > _mappedMemory) { + _mappedMemory = executorMetrics.mappedMemory updated = true } @@ -84,23 +115,13 @@ private[spark] class PeakExecutorMetrics { * values set. */ def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = { - if (jvmUsedMemory < 0) { + if (_jvmUsedHeapMemory < 0) { None } else { - Some(new PeakMemoryMetrics(jvmUsedMemory, onHeapExecutionMemory, - offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory, - onHeapUnifiedMemory, offHeapUnifiedMemory)) + Some(new PeakMemoryMetrics(_jvmUsedHeapMemory, _jvmUsedNonHeapMemory, + _onHeapExecutionMemory, _offHeapExecutionMemory, _onHeapStorageMemory, + _offHeapStorageMemory, _onHeapUnifiedMemory, _offHeapUnifiedMemory, + _directMemory, _mappedMemory)) } } - - /** Clears/resets the saved peak values. */ - def reset(): Unit = { - jvmUsedMemory = -1L; - onHeapExecutionMemory = 0L - offHeapExecutionMemory = 0L - onHeapStorageMemory = 0L - offHeapStorageMemory = 0L - onHeapUnifiedMemory = 0L - offHeapUnifiedMemory = 0L - } } 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 afb3e60e0b28..c296f55210f2 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -269,7 +269,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE def hasMemoryInfo: Boolean = totalOnHeap >= 0L // peak values for executor level metrics - var peakExecutorMetrics = new PeakExecutorMetrics + val peakExecutorMetrics = new PeakExecutorMetrics def hostname: String = if (host != null) host else hostPort.split(":")(0) 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 4d14ba7560e8..15f83213bb6e 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 @@ -109,13 +109,16 @@ class MemoryMetrics private[spark]( val totalOffHeapStorageMemory: Long) class PeakMemoryMetrics private[spark]( - val jvmUsedMemory: Long, + val jvmUsedHeapMemory: Long, + val jvmUsedNonHeapMemory: Long, val onHeapExecutionMemory: Long, val offHeapExecutionMemory: Long, val onHeapStorageMemory: Long, val offHeapStorageMemory: Long, val onHeapUnifiedMemory: Long, - val offHeapUnifiedMemory: Long) + val offHeapUnifiedMemory: Long, + val directMemory: Long, + val mappedMemory: Long) class JobData private[spark]( val jobId: Int, 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 18f5275b0af7..35b6aeb8f0e7 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -390,11 +390,16 @@ private[spark] object JsonProtocol { */ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { ("Timestamp" -> executorMetrics.timestamp) ~ - ("JVM Used Memory" -> executorMetrics.jvmUsedMemory) ~ + ("JVM Used Heap Memory" -> executorMetrics.jvmUsedHeapMemory) ~ + ("JVM Used Nonheap Memory" -> executorMetrics.jvmUsedNonHeapMemory) ~ ("Onheap Execution Memory" -> executorMetrics.onHeapExecutionMemory) ~ ("Offheap Execution Memory" -> executorMetrics.offHeapExecutionMemory) ~ ("Onheap Storage Memory" -> executorMetrics.onHeapStorageMemory) ~ - ("Offheap Storage Memory" -> executorMetrics.offHeapStorageMemory) + ("Offheap Storage Memory" -> executorMetrics.offHeapStorageMemory) ~ + ("Onheap Unified Memory" -> executorMetrics.onHeapUnifiedMemory) ~ + ("Offheap Unified Memory" -> executorMetrics.offHeapUnifiedMemory) ~ + ("Direct Memory" -> executorMetrics.directMemory) ~ + ("Mapped Memory" -> executorMetrics.mappedMemory) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -609,13 +614,20 @@ private[spark] object JsonProtocol { */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val timeStamp = (json \ "Timestamp").extract[Long] - val jvmUsedMemory = (json \ "JVM Used Memory").extract[Long] + val jvmUsedHeapMemory = (json \ "JVM Used Heap Memory").extract[Long] + val jvmUsedNonHeapMemory = (json \ "JVM Used Nonheap Memory").extract[Long] val onHeapExecutionMemory = (json \ "Onheap Execution Memory").extract[Long] val offHeapExecutionMemory = (json \ "Offheap Execution Memory").extract[Long] val onHeapStorageMemory = (json \ "Onheap Storage Memory").extract[Long] val offHeapStorageMemory = (json \ "Offheap Storage Memory").extract[Long] - new ExecutorMetrics(timeStamp, jvmUsedMemory, onHeapExecutionMemory, - offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory) + val onHeapUnifiedMemory = (json \ "Onheap Unified Memory").extract[Long] + val offHeapUnifiedMemory = (json \ "Offheap Unified Memory").extract[Long] + val directMemory = (json \ "Direct Memory").extract[Long] + val mappedMemory = (json \ "Mapped Memory").extract[Long] + new ExecutorMetrics(timeStamp, jvmUsedHeapMemory, jvmUsedNonHeapMemory, + onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, + offHeapStorageMemory, onHeapUnifiedMemory, offHeapUnifiedMemory, directMemory, + mappedMemory) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { 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 9858c73fae2f..eea6f595efd2 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -2,16 +2,16 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501 + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134 } ] }, { "id" : "application_1516285256255_0012", 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 94eddfa0c7bb..7bc7f31be097 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 @@ -2,15 +2,15 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501, + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134, "lastUpdatedEpoch" : 0 } ] }, { 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 1cd5fbb794cb..ac1bb97b157f 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 @@ -1,6 +1,6 @@ [ { "id" : "driver", - "hostPort" : "cluster-node0033.company.com:53121", + "hostPort" : "node0033.grid.company.com:60749", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -17,28 +17,31 @@ "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : false, - "maxMemory" : 1046269132, - "addTime" : "2018-03-19T18:21:24.751GMT", + "maxMemory" : 1043437977, + "addTime" : "2018-04-19T23:55:05.107GMT", "executorLogs" : { }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 1046269132, + "totalOnHeapStorageMemory" : 1043437977, "totalOffHeapStorageMemory" : 0 }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 671982856, + "jvmUsedHeapMemory" : 629553808, + "jvmUsedNonHeapMemory" : 205304696, "onHeapExecutionMemory" : 0, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 900126, + "onHeapStorageMemory" : 905801, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 900126, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 905801, + "offHeapUnifiedMemory" : 0, + "directMemory" : 397602, + "mappedMemory" : 0 } }, { - "id" : "8", - "hostPort" : "cluster-node6128.company.com:1664", + "id" : "7", + "hostPort" : "node6340.grid.company.com:5933", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -56,51 +59,10 @@ "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:22:25.988GMT", - "executorLogs" : { - "stdout" : "http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 956615884, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "jvmUsedMemory" : 226589176, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 0, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 0, - "offHeapUnifiedMemory" : 0 - } -}, { - "id" : "7", - "hostPort" : "cluster-node2449.company.com:26914", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 1, - "maxTasks" : 1, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 11073, - "totalGCTime" : 942, - "totalInputBytes" : 36849246, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 349006, - "isBlacklisted" : false, - "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:22:10.182GMT", + "addTime" : "2018-04-19T23:55:49.826GMT", "executorLogs" : { - "stdout" : "http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stderr?start=-4096" + "stdout" : "http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096", + "stderr" : "http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -108,19 +70,10 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "jvmUsedMemory" : 365803960, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 562630, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 562630, - "offHeapUnifiedMemory" : 0 - } + "blacklistedInStages" : [ ] }, { "id" : "6", - "hostPort" : "cluster-node6148.company.com:1233", + "hostPort" : "node6644.grid.company.com:8445", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -129,19 +82,19 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 34956, - "totalGCTime" : 5484, - "totalInputBytes" : 24250210, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, "totalShuffleRead" : 0, - "totalShuffleWrite" : 242714, + "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:22:19.537GMT", + "addTime" : "2018-04-19T23:55:47.549GMT", "executorLogs" : { - "stdout" : "http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stderr?start=-4096" + "stdout" : "http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stdout?start=-4096", + "stderr" : "http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -149,19 +102,10 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "jvmUsedMemory" : 501950360, - "onHeapExecutionMemory" : 35651584, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 562630, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 36214214, - "offHeapUnifiedMemory" : 0 - } + "blacklistedInStages" : [ ] }, { "id" : "5", - "hostPort" : "cluster-node5882.company.com:24590", + "hostPort" : "node2477.grid.company.com:20123", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -172,17 +116,17 @@ "failedTasks" : 0, "completedTasks" : 1, "totalTasks" : 1, - "totalDuration" : 30945, - "totalGCTime" : 6111, + "totalDuration" : 9252, + "totalGCTime" : 920, "totalInputBytes" : 36838295, "totalShuffleRead" : 0, "totalShuffleWrite" : 355051, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:22:11.267GMT", + "addTime" : "2018-04-19T23:55:43.160GMT", "executorLogs" : { - "stdout" : "http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stderr?start=-4096" + "stdout" : "http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stdout?start=-4096", + "stderr" : "http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -190,19 +134,10 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "jvmUsedMemory" : 513138800, - "onHeapExecutionMemory" : 41943040, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 562630, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 42505670, - "offHeapUnifiedMemory" : 0 - } + "blacklistedInStages" : [ ] }, { "id" : "4", - "hostPort" : "cluster-node6137.company.com:19036", + "hostPort" : "node4243.grid.company.com:16084", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -211,19 +146,19 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 79365, - "totalGCTime" : 6979, - "totalInputBytes" : 50409514, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 31362123, + "completedTasks" : 3, + "totalTasks" : 3, + "totalDuration" : 15645, + "totalGCTime" : 405, + "totalInputBytes" : 87272855, + "totalShuffleRead" : 438675, + "totalShuffleWrite" : 26773039, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:21:44.950GMT", + "addTime" : "2018-04-19T23:55:12.278GMT", "executorLogs" : { - "stdout" : "http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stderr?start=-4096" + "stdout" : "http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stdout?start=-4096", + "stderr" : "http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -233,17 +168,20 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 707255176, - "onHeapExecutionMemory" : 0, + "jvmUsedHeapMemory" : 518613056, + "jvmUsedNonHeapMemory" : 95657456, + "onHeapExecutionMemory" : 37748736, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 83580552, + "onHeapStorageMemory" : 63104457, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 83580552, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 100853193, + "offHeapUnifiedMemory" : 0, + "directMemory" : 126261, + "mappedMemory" : 0 } }, { "id" : "3", - "hostPort" : "cluster-node1177.company.com:12614", + "hostPort" : "node0998.grid.company.com:45265", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -252,19 +190,19 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 2, - "totalTasks" : 2, - "totalDuration" : 38198, - "totalGCTime" : 1024, - "totalInputBytes" : 50423609, - "totalShuffleRead" : 438675, - "totalShuffleWrite" : 26424033, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 14491, + "totalGCTime" : 342, + "totalInputBytes" : 50409514, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 31362123, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:21:33.370GMT", + "addTime" : "2018-04-19T23:55:12.088GMT", "executorLogs" : { - "stdout" : "http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stderr?start=-4096" + "stdout" : "http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stdout?start=-4096", + "stderr" : "http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -274,17 +212,20 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 620936920, + "jvmUsedHeapMemory" : 726805712, + "jvmUsedNonHeapMemory" : 90709624, "onHeapExecutionMemory" : 0, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 72624976, + "onHeapStorageMemory" : 69535048, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 72624976, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 69535048, + "offHeapUnifiedMemory" : 0, + "directMemory" : 87796, + "mappedMemory" : 0 } }, { "id" : "2", - "hostPort" : "cluster-node6036.company.com:17435", + "hostPort" : "node4045.grid.company.com:29262", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -295,17 +236,17 @@ "failedTasks" : 0, "completedTasks" : 1, "totalTasks" : 1, - "totalDuration" : 44061, - "totalGCTime" : 4677, + "totalDuration" : 14113, + "totalGCTime" : 326, "totalInputBytes" : 50423423, "totalShuffleRead" : 0, "totalShuffleWrite" : 22950296, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:21:41.213GMT", + "addTime" : "2018-04-19T23:55:12.471GMT", "executorLogs" : { - "stdout" : "http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stderr?start=-4096" + "stdout" : "http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stdout?start=-4096", + "stderr" : "http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -315,17 +256,20 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 695372944, + "jvmUsedHeapMemory" : 595946552, + "jvmUsedNonHeapMemory" : 91208368, "onHeapExecutionMemory" : 0, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 58468552, + "onHeapStorageMemory" : 58468944, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 58468552, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 58468944, + "offHeapUnifiedMemory" : 0, + "directMemory" : 87796, + "mappedMemory" : 0 } }, { "id" : "1", - "hostPort" : "cluster-node4342.company.com:29144", + "hostPort" : "node1404.grid.company.com:34043", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -334,19 +278,19 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 2, - "totalTasks" : 2, - "totalDuration" : 31501, - "totalGCTime" : 936, - "totalInputBytes" : 74654808, + "completedTasks" : 3, + "totalTasks" : 3, + "totalDuration" : 15665, + "totalGCTime" : 471, + "totalInputBytes" : 98905018, "totalShuffleRead" : 0, - "totalShuffleWrite" : 20352030, + "totalShuffleWrite" : 20594744, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:21:33.307GMT", + "addTime" : "2018-04-19T23:55:11.695GMT", "executorLogs" : { - "stdout" : "http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stderr?start=-4096" + "stdout" : "http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stdout?start=-4096", + "stderr" : "http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -356,12 +300,15 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 505317000, - "onHeapExecutionMemory" : 35651584, + "jvmUsedHeapMemory" : 755008624, + "jvmUsedNonHeapMemory" : 100519936, + "onHeapExecutionMemory" : 0, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 47961406, + "onHeapStorageMemory" : 47962185, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 83612990, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 47962185, + "offHeapUnifiedMemory" : 0, + "directMemory" : 98230, + "mappedMemory" : 0 } } ] 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 bffe37ce52b4..9e1e65a35881 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 @@ -2,16 +2,16 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501 + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134 } ] }, { "id" : "application_1516285256255_0012", 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 066217dacbe9..28c6bf1b3e01 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 @@ -2,16 +2,16 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501 + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134 } ] }, { "id" : "application_1516285256255_0012", 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 b1226c905d6a..f547b79f47e1 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 @@ -2,16 +2,16 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501 + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134 } ] }, { "id" : "application_1516285256255_0012", diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 index 26289113d369..b3424f9582c0 100644 --- a/core/src/test/resources/spark-events/application_1506645932520_24630151 +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -1,106 +1,63 @@ {"Event":"SparkListenerLogStart","Spark Version":"2.4.0-SNAPSHOT"} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483677960,"JVM Used Memory":587908264,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"cluster-node0033.company.com","Port":53121},"Maximum Memory":1046269132,"Timestamp":1521483684751,"Maximum Onheap Memory":1046269132,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_31/jre","Java Version":"1.8.0_31 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.jars.ivySettings":"/export/apps/spark/commonconf/ivysettings.xml","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.driver.host":"cluster-node0033.company.com","spark.dynamicAllocation.sustainedSchedulerBacklogTimeout":"5","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"36653","spark.shuffle.service.enabled":"true","spark.ui.acls.enable":"true","spark.reducer.maxSizeInFlight":"48m","spark.yarn.queue":"spark_default","spark.repl.class.uri":"spark://cluster-node0033.company.com:36653/classes","spark.jars":"","spark.yarn.historyServer.address":"clustersh01.company.com:18080","spark.memoryOverhead.multiplier.percent":"10","spark.repl.class.outputDir":"/grid/a/mapred/tmp/spark-c015ad60-cb0b-4920-a14d-e79aa91000f3/repl-c23128a3-a417-40f7-933e-3cd0a5e7c747","spark.dynamicAllocation.cachedExecutorIdleTimeout":"1200","spark.yarn.access.namenodes":"hdfs://clusternn02.company.com:9000","spark.app.name":"Spark shell","spark.dynamicAllocation.schedulerBacklogTimeout":"5","spark.yarn.security.credentials.hive.enabled":"false","spark.yarn.am.cores":"1","spark.memoryOverhead.min":"384","spark.scheduler.mode":"FIFO","spark.driver.memory":"2G","spark.executor.instances":"4","spark.isolated.classloader.additional.classes.prefix":"com_company_","spark.logConf":"true","spark.ui.showConsoleProgress":"true","spark.user.priority.jars":"*********(redacted)","spark.isolated.classloader":"true","spark.sql.sources.schemaStringLengthThreshold":"40000","spark.yarn.secondary.jars":"spark-avro_2.11-3.2.0.21.jar,grid-topology-1.0.jar","spark.reducer.maxBlocksInFlightPerAddress":"100","spark.dynamicAllocation.maxExecutors":"900","spark.yarn.appMasterEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.executor.id":"driver","spark.yarn.am.memory":"2G","spark.driver.cores":"1","spark.min.mem.vore.ratio":"5","spark.sql.sources.partitionOverwriteMode":"DYNAMIC","spark.submit.deployMode":"client","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.isolated.classloader.default.jar":"*dali-data-spark*","spark.authenticate":"true","spark.eventLog.usexattr":"true","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.memory":"2G","spark.home":"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51","spark.reducer.maxReqsInFlight":"10","spark.eventLog.dir":"hdfs://clusternn02.company.com:9000/system/spark-history","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.isolated.classes":"org.apache.hadoop.hive.ql.io.CombineHiveInputFormat$CombineHiveInputSplit","spark.eventLog.compress":"true","spark.executor.cores":"1","spark.version":"2.1.0","spark.driver.appUIAddress":"http://cluster-node0033.company.com:46527","spark.repl.local.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"clusterwp01.company.com","spark.min.memory-gb.size":"10","spark.dynamicAllocation.minExecutors":"1","spark.dynamicAllocation.initialExecutors":"3","spark.expressionencoder.org.apache.avro.specific.SpecificRecord":"com.databricks.spark.avro.AvroEncoder$","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://clusterwp01.company.com:8080/proxy/application_1506645932520_24630151","spark.executorEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.dynamicAllocation.executorIdleTimeout":"150","spark.shell.auto.node.labeling":"true","spark.yarn.dist.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.app.id":"application_1506645932520_24630151","spark.ui.view.acls":"*"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_31/jre/lib/amd64","user.dir":"*********(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.31-b07","java.endorsed.dirs":"/usr/java/jdk1.8.0_31/jre/lib/endorsed","java.runtime.version":"1.8.0_31-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_31/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_31/jre/lib/resources.jar:/usr/java/jdk1.8.0_31/jre/lib/rt.jar:/usr/java/jdk1.8.0_31/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_31/jre/lib/jsse.jar:/usr/java/jdk1.8.0_31/jre/lib/jce.jar:/usr/java/jdk1.8.0_31/jre/lib/charsets.jar:/usr/java/jdk1.8.0_31/jre/lib/jfr.jar:/usr/java/jdk1.8.0_31/jre/classes","file.encoding":"ANSI_X3.4-1968","user.timezone":"*********(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"2.6.32-504.16.2.el6.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*********(redacted)","sun.jnu.encoding":"ANSI_X3.4-1968","user.language":"*********(redacted)","java.vendor.url":"*********(redacted)","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":"*********(redacted)","user.name":"*********(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --class org.apache.spark.repl.Main --name Spark shell --jars /export/home/edlu/spark-avro_2.11-3.2.0.21.jar,/export/apps/hadoop/site/lib/grid-topology-1.0.jar --num-executors 4 spark-shell","java.home":"/usr/java/jdk1.8.0_31/jre","java.version":"1.8.0_31","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-servlet-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/derby-10.12.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-reflect-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-graphx_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-util-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/base64-2.3.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-auth-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/validation-api-1.1.0.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-api-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/objenesis-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/conf/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpclient-4.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/kryo-shaded-3.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-library-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-net-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xz-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-annotations-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/activation-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arpack_combined_all-0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libthrift-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aircompressor-0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-jackson-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/asm-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-hive_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ivy-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/export/apps/hadoop/site/etc/hadoop/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-java-1.1.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-format-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-all-4.1.17.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-ipc-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xmlenc-0.52.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jdo-api-3.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-client-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-runtime-3.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/pyrolite-4.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-catalyst_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-collections-3.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-api-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stream-2.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-format-2.3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-vector-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-web-proxy-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sketch_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-common-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hppc-0.7.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sql_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/univocity-parsers-2.5.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-math3-3.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compiler-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-1.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/java-xmlbuilder-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-annotations-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-3.9.9.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zookeeper-3.4.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-compiler-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/eigenbase-properties-1.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-yarn_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/JavaEWAH-0.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsr305-1.3.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libfb303-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.annotation-api-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-digester-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-jvm-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-framework-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/paranamer-2.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/janino-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-core-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-core-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsch-0.1.42.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-unsafe_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-codec-1.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jtransforms-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/lz4-java-1.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-core-3.2.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/core-1.1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/leveldbjni-all-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-dbcp-1.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang3-3.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill-java-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jodd-core-3.5.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-pool-1.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/minlog-1.3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/gson-2.2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/py4j-0.10.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-streaming_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/machinist_2.11-0.6.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-0.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-app-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-graphite-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-core-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib-local_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-memory-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-guava-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xercesImpl-2.9.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-tags_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javolution-5.5.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/joda-time-2.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-2.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-jobclient-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang-2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/compress-lzf-1.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-crypto-1.0.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-core-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-recipes-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guava-14.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-core_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-common_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-launcher_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr4-runtime-4.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-util-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jaxb-api-2.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-io-2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-encoding-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpcore-4.4.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-xc-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/protobuf-java-2.5.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-configuration-1.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compress-1.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-mapreduce-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ST4-4.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-core-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-shuffle-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-repl_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/opencsv-2.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-logging-1.1.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-cli-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-client-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-hdfs-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/log4j-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-column-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill_2.11-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stringtemplate-3.2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-common-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-shuffle_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-kvstore_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jta-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javassist-3.18.1-GA.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-httpclient-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jets3t-0.9.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-json-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bcprov-jdk15on-1.58.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/oro-2.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsp-api-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1506645932520_24630151","Timestamp":1521483660843,"User":"edlu"} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483693307,"Executor ID":"1","Executor Info":{"Host":"cluster-node4342.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stdout?start=-4096","stderr":"http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483693370,"Executor ID":"3","Executor Info":{"Host":"cluster-node1177.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stdout?start=-4096","stderr":"http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"cluster-node4342.company.com","Port":29144},"Maximum Memory":956615884,"Timestamp":1521483693405,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"cluster-node1177.company.com","Port":12614},"Maximum Memory":956615884,"Timestamp":1521483693483,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483701213,"Executor ID":"2","Executor Info":{"Host":"cluster-node6036.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stdout?start=-4096","stderr":"http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"cluster-node6036.company.com","Port":17435},"Maximum Memory":956615884,"Timestamp":1521483701726,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483704950,"Executor ID":"4","Executor Info":{"Host":"cluster-node6137.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stdout?start=-4096","stderr":"http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"cluster-node6137.company.com","Port":19036},"Maximum Memory":956615884,"Timestamp":1521483705369,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line44.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line44.$read$$iw$$iw$$iw$$iw.(:59)\n$line44.$read$$iw$$iw$$iw.(:61)\n$line44.$read$$iw$$iw.(:63)\n$line44.$read$$iw.(:65)\n$line44.$read.(:67)\n$line44.$read$.(:71)\n$line44.$read$.()\n$line44.$eval$.$print$lzycompute(:7)\n$line44.$eval$.$print(:6)\n$line44.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1521483706188} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1521483706193} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483707958,"JVM Used Memory":511901456,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":415523,"Offheap Storage Memory":0}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line48.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line48.$read$$iw$$iw$$iw$$iw.(:59)\n$line48.$read$$iw$$iw$$iw.(:61)\n$line48.$read$$iw$$iw.(:63)\n$line48.$read$$iw.(:65)\n$line48.$read.(:67)\n$line48.$read$.(:71)\n$line48.$read$.()\n$line48.$eval$.$print$lzycompute(:7)\n$line48.$eval$.$print(:6)\n$line48.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1521483708914} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1521483708914} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483712396,"JVM Used Memory":197797104,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at :40","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- AnalysisBarrier\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nappId: string, attemptId: string, name: string, mode: string, completed: string, duration: string, endTime: string, endTimeEpoch: string, lastUpdated: string, lastUpdatedEpoch: string, sparkUser: string, startTime: string, startTimeEpoch: string, appSparkVersion: string, endDate: string, azkaban.link.workflow.url: string, azkaban.link.execution.url: string, azkaban.link.job.url: string, user.name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@babef5e, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) LocalLimit 21\n +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@babef5e, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"LocalLimit","simpleString":"LocalLimit 21","children":[{"nodeName":"Project","simpleString":"Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"InMemoryTableScan","simpleString":"InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]","children":[],"metrics":[{"name":"number of output rows","accumulatorId":35,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":36,"metricType":"timing"}]}],"metrics":[]}],"metrics":[]}],"metrics":[]}],"metrics":[{"name":"duration total (min, med, max)","accumulatorId":34,"metricType":"timing"}]}],"metrics":[]},"time":1521483710146} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1521483710385,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]}],"Stage IDs":[0,1,2],"Properties":{"spark.sql.execution.id":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710427,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710514,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1521483710517,"Executor ID":"2","Host":"cluster-node6036.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"4","Host":"cluster-node6137.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483715514,"JVM Used Memory":83495872,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":36413,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483720047,"JVM Used Memory":121617408,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":36413,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483717971,"JVM Used Memory":603050152,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483722392,"JVM Used Memory":256639776,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483724389,"JVM Used Memory":431714816,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483725467,"JVM Used Memory":504156608,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":25165398,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483730046,"JVM Used Memory":292871096,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483727958,"JVM Used Memory":607069712,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1521483728324,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483728367,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"154334487","Value":"154334486","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"27714","Value":"27713","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":37809697,"Value":37809697,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":98699361,"Value":98699361,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":20002743,"Value":20002743,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":834,"Value":834,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":1856,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":13106272441,"Value":13106272441,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":15665,"Value":15665,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":741777109,"Value":741777109,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":2081,"Value":2081,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2081,"Executor Deserialize CPU Time":741777109,"Executor Run Time":15665,"Executor CPU Time":13106272441,"Result Size":1856,"JVM GC Time":834,"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":20002743,"Shuffle Write Time":98699361,"Shuffle Records Written":466636},"Input Metrics":{"Bytes Read":37809697,"Records Read":466636},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483732392,"JVM Used Memory":207788120,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":21875994,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483730182,"Executor ID":"7","Executor Info":{"Host":"cluster-node2449.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stdout?start=-4096","stderr":"http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1521483730192,"Executor ID":"7","Host":"cluster-node2449.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"cluster-node2449.company.com","Port":26914},"Maximum Memory":956615884,"Timestamp":1521483730295,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483731267,"Executor ID":"5","Executor Info":{"Host":"cluster-node5882.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stdout?start=-4096","stderr":"http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1521483731270,"Executor ID":"5","Host":"cluster-node5882.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"cluster-node5882.company.com","Port":24590},"Maximum Memory":956615884,"Timestamp":1521483731726,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483734387,"JVM Used Memory":585461008,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483735466,"JVM Used Memory":286822456,"Onheap Execution Memory":35651584,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483737958,"JVM Used Memory":619161088,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483739537,"Executor ID":"6","Executor Info":{"Host":"cluster-node6148.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stdout?start=-4096","stderr":"http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1521483739540,"Executor ID":"6","Host":"cluster-node6148.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"cluster-node6148.company.com","Port":1233},"Maximum Memory":956615884,"Timestamp":1521483739859,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483742392,"JVM Used Memory":618421104,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":72624976,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1521483730192,"Executor ID":"7","Host":"cluster-node2449.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483741265,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"1953294","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"3575","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"76","Value":"75","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"7964","Value":"7963","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"196587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"3575","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"7495","Value":"7494","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":3575,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":36849246,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":71122623,"Value":71122623,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":3575,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":349006,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":942,"Value":942,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":2437,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5608539520,"Value":5608539520,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":9646,"Value":9646,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":775466204,"Value":775466204,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1177,"Value":1177,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1177,"Executor Deserialize CPU Time":775466204,"Executor Run Time":9646,"Executor CPU Time":5608539520,"Result Size":2437,"JVM GC Time":942,"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":349006,"Shuffle Write Time":71122623,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483744387,"JVM Used Memory":533322584,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":52049755,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1521483728324,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483741988,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"63","Value":"138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"13344","Value":"21307","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"12907","Value":"20401","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":43359041,"Value":114481664,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":102,"Value":1044,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4831,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":2885767088,"Value":8494306608,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":13538,"Value":23184,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":90660797,"Value":866127001,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":104,"Value":1281,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":104,"Executor Deserialize CPU Time":90660797,"Executor Run Time":13538,"Executor CPU Time":2885767088,"Result Size":2394,"JVM GC Time":102,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":43359041,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483745464,"JVM Used Memory":504693976,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483746631,"JVM Used Memory":365802552,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483745988,"Executor ID":"8","Executor Info":{"Host":"cluster-node6128.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stdout?start=-4096","stderr":"http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"8","Host":"cluster-node6128.company.com","Port":1664},"Maximum Memory":956615884,"Timestamp":1521483746284,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483746534,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"206421303","Value":"360755789","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"65190","Value":"92903","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423609,"Value":88233306,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":123692332,"Value":222391693,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":26424033,"Value":46426776,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":878,"Value":1712,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":3712,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":19065776829,"Value":32172049270,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":35010,"Value":50675,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":732371255,"Value":1474148364,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":942,"Value":3023,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":942,"Executor Deserialize CPU Time":732371255,"Executor Run Time":35010,"Executor CPU Time":19065776829,"Result Size":1856,"JVM GC Time":878,"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":26424033,"Shuffle Write Time":123692332,"Shuffle Records Written":624246},"Input Metrics":{"Bytes Read":50423609,"Records Read":624246},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483750045,"JVM Used Memory":207536536,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":2461716,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483747958,"JVM Used Memory":641729640,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"5","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483751931,"JVM Used Memory":513138800,"Onheap Execution Memory":65536,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483752392,"JVM Used Memory":619676000,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483754387,"JVM Used Memory":695372944,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468552,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483755464,"JVM Used Memory":504694360,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483756630,"JVM Used Memory":365802904,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1521483710517,"Executor ID":"2","Host":"cluster-node6036.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483754578,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"73812","Value":"166715","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":214358240,"Value":436749933,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":3,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":4677,"Value":6389,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":24023445128,"Value":56195494398,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":42072,"Value":92747,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":952244001,"Value":2426392365,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":1796,"Value":4819,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1796,"Executor Deserialize CPU Time":952244001,"Executor Run Time":42072,"Executor CPU Time":24023445128,"Result Size":1856,"JVM GC Time":4677,"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":22950296,"Shuffle Write Time":214358240,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"6","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483758435,"JVM Used Memory":397749936,"Onheap Execution Memory":65536,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483760046,"JVM Used Memory":488354552,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":25910376,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483757958,"JVM Used Memory":662984224,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483762392,"JVM Used Memory":619676384,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"5","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483762915,"JVM Used Memory":353487552,"Onheap Execution Memory":41943040,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1521483731270,"Executor ID":"5","Host":"cluster-node5882.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483762215,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"5784372","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"10678","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"125829116","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"131","Value":"269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"22823","Value":"44130","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"586742","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"10679","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"19841","Value":"40242","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":10679,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":110532652,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":215191844,"Value":329673508,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":10678,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1053344,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":125829120,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":9,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":6111,"Value":7155,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":7268,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":8017651595,"Value":16511958203,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":27779,"Value":50963,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1058013355,"Value":1924140356,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":2357,"Value":3638,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2357,"Executor Deserialize CPU Time":1058013355,"Executor Run Time":27779,"Executor CPU Time":8017651595,"Result Size":2437,"JVM GC Time":6111,"Result Serialization Time":9,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355051,"Shuffle Write Time":215191844,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483765464,"JVM Used Memory":504694712,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483766630,"JVM Used Memory":365803256,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483767180,"JVM Used Memory":226589112,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"6","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483768433,"JVM Used Memory":501950360,"Onheap Execution Memory":35651584,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483770047,"JVM Used Memory":180202344,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":55709919,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483767958,"JVM Used Memory":665764240,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483772392,"JVM Used Memory":619676736,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483775464,"JVM Used Memory":504695064,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483776630,"JVM Used Memory":365803608,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1521483739540,"Executor ID":"6","Host":"cluster-node6148.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483774496,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"300","Value":"569","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"26291","Value":"70421","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"24092","Value":"64334","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":521474789,"Value":851148297,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":13,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":5484,"Value":12639,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9705,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":7239844597,"Value":23751802800,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":30603,"Value":81566,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1062050912,"Value":2986191268,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3790,"Value":7428,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3790,"Executor Deserialize CPU Time":1062050912,"Executor Run Time":30603,"Executor CPU Time":7239844597,"Result Size":2437,"JVM GC Time":5484,"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":242714,"Shuffle Write Time":521474789,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710514,"Completion Time":1521483774499,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":7428,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":23751802800,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":81566,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":851148297,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":12639,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2986191268,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"70421","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9705,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"569","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"64334","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483777175,"JVM Used Memory":226589144,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483780046,"JVM Used Memory":639104112,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":83580552,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483777958,"JVM Used Memory":669979432,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483782392,"JVM Used Memory":620936568,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483785464,"JVM Used Memory":505317000,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483786630,"JVM Used Memory":365803960,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483787175,"JVM Used Memory":226589176,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483790047,"JVM Used Memory":707255176,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69534656,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483787958,"JVM Used Memory":671982856,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483792392,"JVM Used Memory":620936920,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"4","Host":"cluster-node6137.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483789895,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"131866","Value":"298581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":464179545,"Value":900929478,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":6979,"Value":13368,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":25321337014,"Value":81516831412,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":74896,"Value":167643,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1173542343,"Value":3599934708,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":4315,"Value":9134,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4315,"Executor Deserialize CPU Time":1173542343,"Executor Run Time":74896,"Executor CPU Time":25321337014,"Result Size":1856,"JVM GC Time":6979,"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":31362123,"Shuffle Write Time":464179545,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710427,"Completion Time":1521483789896,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3599934708,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"298581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":81516831412,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":9134,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":900929478,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":167643,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":13368,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483789907,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1521483789924,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483794387,"JVM Used Memory":317344968,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468552,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1521483789924,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1521483792118,"Failed":false,"Killed":false,"Accumulables":[{"ID":34,"Name":"duration total (min, med, max)","Update":"1","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":35,"Name":"number of output rows","Update":"6928","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Update":"1181","Value":"1180","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"number of output rows","Update":"10945","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"number of output rows","Update":"62","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Update":"33619967","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"sort time total (min, med, max)","Update":"6","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":22,"Name":"duration total (min, med, max)","Update":"896","Value":"895","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"peak memory total (min, med, max)","Update":"34078719","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Update":"53","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"duration total (min, med, max)","Update":"1109","Value":"1108","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Update":11007,"Value":11007,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":122827,"Value":122827,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":315848,"Value":315848,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":7,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Update":67698688,"Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":92,"Name":"internal.metrics.jvmGCTime","Update":146,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Update":4685,"Value":4685,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Update":970272606,"Value":970272606,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.executorRunTime","Update":1923,"Value":1923,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Update":162560302,"Value":162560302,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Update":230,"Value":230,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":230,"Executor Deserialize CPU Time":162560302,"Executor Run Time":1923,"Executor CPU Time":970272606,"Result Size":4685,"JVM GC Time":146,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":7,"Local Blocks Fetched":1,"Fetch Wait Time":1,"Remote Bytes Read":315848,"Remote Bytes Read To Disk":0,"Local Bytes Read":122827,"Total Records Read":11007},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":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":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483789907,"Completion Time":1521483792119,"Accumulables":[{"ID":92,"Name":"internal.metrics.jvmGCTime","Value":146,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Value":11007,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"number of output rows","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":89,"Name":"internal.metrics.executorRunTime","Value":1923,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":7,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"number of output rows","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":315848,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Value":4685,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"peak memory total (min, med, max)","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"duration total (min, med, max)","Value":"895","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Value":162560302,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"duration total (min, med, max)","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Value":"1180","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"sort time total (min, med, max)","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Value":230,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Value":970272606,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"number of output rows","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":122827,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"duration total (min, med, max)","Value":"1108","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1521483792127,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1521483792153} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node0033.grid.company.com","Port":60749},"Maximum Memory":1043437977,"Timestamp":1524182105107,"Maximum Onheap Memory":1043437977,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_31/jre","Java Version":"1.8.0_31 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.jars.ivySettings":"/export/apps/spark/commonconf/ivysettings.xml","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.driver.host":"node0033.grid.company.com","spark.dynamicAllocation.sustainedSchedulerBacklogTimeout":"5","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"57705","spark.shuffle.service.enabled":"true","spark.ui.acls.enable":"true","spark.reducer.maxSizeInFlight":"48m","spark.yarn.queue":"spark_default","spark.repl.class.uri":"spark://node0033.grid.company.com:57705/classes","spark.jars":"","spark.yarn.historyServer.address":"clustersh01.grid.company.com:18080","spark.memoryOverhead.multiplier.percent":"10","spark.repl.class.outputDir":"/grid/a/mapred/tmp/spark-21b68b4b-c1db-460e-a228-b87545d870f1/repl-58778a76-04c1-434d-bfb7-9a9b83afe718","spark.dynamicAllocation.cachedExecutorIdleTimeout":"1200","spark.yarn.access.namenodes":"hdfs://clusternn02.grid.company.com:9000","spark.app.name":"Spark shell","spark.dynamicAllocation.schedulerBacklogTimeout":"5","spark.yarn.security.credentials.hive.enabled":"false","spark.yarn.am.cores":"1","spark.memoryOverhead.min":"384","spark.scheduler.mode":"FIFO","spark.driver.memory":"2G","spark.executor.instances":"4","spark.isolated.classloader.additional.classes.prefix":"com_company_","spark.logConf":"true","spark.ui.showConsoleProgress":"true","spark.user.priority.jars":"*********(redacted)","spark.isolated.classloader":"true","spark.sql.sources.schemaStringLengthThreshold":"40000","spark.yarn.secondary.jars":"spark-avro_2.11-3.2.0.21.jar,grid-topology-1.0.jar","spark.reducer.maxBlocksInFlightPerAddress":"100","spark.dynamicAllocation.maxExecutors":"900","spark.yarn.appMasterEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.executor.id":"driver","spark.yarn.am.memory":"2G","spark.driver.cores":"1","spark.search.packages":"com.company.dali:dali-data-spark,com.company.spark-common:spark-common","spark.min.mem.vore.ratio":"5","spark.sql.sources.partitionOverwriteMode":"DYNAMIC","spark.submit.deployMode":"client","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.default.packages":"com.company.dali:dali-data-spark:8.+?classifier=all,com.company.spark-common:spark-common_2.10:0.+?","spark.isolated.classloader.default.jar":"*dali-data-spark*","spark.authenticate":"true","spark.eventLog.usexattr":"true","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.memory":"2G","spark.home":"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51","spark.reducer.maxReqsInFlight":"10","spark.eventLog.dir":"hdfs://clusternn02.grid.company.com:9000/system/spark-history","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.isolated.classes":"org.apache.hadoop.hive.ql.io.CombineHiveInputFormat$CombineHiveInputSplit","spark.eventLog.compress":"true","spark.executor.cores":"1","spark.version":"2.1.0","spark.driver.appUIAddress":"http://node0033.grid.company.com:8364","spark.repl.local.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"clusterwp01.grid.company.com","spark.min.memory-gb.size":"10","spark.dynamicAllocation.minExecutors":"1","spark.dynamicAllocation.initialExecutors":"3","spark.expressionencoder.org.apache.avro.specific.SpecificRecord":"com.databricks.spark.avro.AvroEncoder$","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://clusterwp01.grid.company.com:8080/proxy/application_1506645932520_24630151","spark.executorEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.dynamicAllocation.executorIdleTimeout":"150","spark.shell.auto.node.labeling":"true","spark.yarn.dist.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.app.id":"application_1506645932520_24630151","spark.ui.view.acls":"*"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_31/jre/lib/amd64","user.dir":"*********(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.31-b07","java.endorsed.dirs":"/usr/java/jdk1.8.0_31/jre/lib/endorsed","java.runtime.version":"1.8.0_31-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_31/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_31/jre/lib/resources.jar:/usr/java/jdk1.8.0_31/jre/lib/rt.jar:/usr/java/jdk1.8.0_31/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_31/jre/lib/jsse.jar:/usr/java/jdk1.8.0_31/jre/lib/jce.jar:/usr/java/jdk1.8.0_31/jre/lib/charsets.jar:/usr/java/jdk1.8.0_31/jre/lib/jfr.jar:/usr/java/jdk1.8.0_31/jre/classes","file.encoding":"UTF-8","user.timezone":"*********(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"2.6.32-504.16.2.el6.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*********(redacted)","sun.jnu.encoding":"UTF-8","user.language":"*********(redacted)","java.vendor.url":"*********(redacted)","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":"*********(redacted)","user.name":"*********(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --class org.apache.spark.repl.Main --name Spark shell --jars /export/home/edlu/spark-avro_2.11-3.2.0.21.jar,/export/apps/hadoop/site/lib/grid-topology-1.0.jar --num-executors 4 spark-shell","java.home":"/usr/java/jdk1.8.0_31/jre","java.version":"1.8.0_31","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-servlet-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/derby-10.12.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-reflect-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-graphx_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-util-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/base64-2.3.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-auth-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/validation-api-1.1.0.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-api-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/objenesis-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/conf/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpclient-4.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/kryo-shaded-3.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-library-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-net-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xz-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-annotations-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/activation-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arpack_combined_all-0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libthrift-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aircompressor-0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-jackson-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/asm-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-hive_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ivy-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/export/apps/hadoop/site/etc/hadoop/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-java-1.1.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-format-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-all-4.1.17.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-ipc-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xmlenc-0.52.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jdo-api-3.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-client-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-runtime-3.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/pyrolite-4.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-catalyst_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-collections-3.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-api-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stream-2.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-format-2.3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-vector-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-web-proxy-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sketch_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-common-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hppc-0.7.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sql_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/univocity-parsers-2.5.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-math3-3.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compiler-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-1.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/java-xmlbuilder-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-annotations-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-3.9.9.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zookeeper-3.4.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-compiler-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/eigenbase-properties-1.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-yarn_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/JavaEWAH-0.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsr305-1.3.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libfb303-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.annotation-api-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-digester-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-jvm-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-framework-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/paranamer-2.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/janino-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-core-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-core-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsch-0.1.42.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-unsafe_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-codec-1.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jtransforms-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/lz4-java-1.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-core-3.2.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/core-1.1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/leveldbjni-all-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-dbcp-1.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang3-3.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill-java-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jodd-core-3.5.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-pool-1.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/minlog-1.3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/gson-2.2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/py4j-0.10.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-streaming_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/machinist_2.11-0.6.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-0.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-app-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-graphite-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-core-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib-local_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-memory-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-guava-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xercesImpl-2.9.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-tags_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javolution-5.5.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/joda-time-2.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-2.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-jobclient-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang-2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/compress-lzf-1.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-crypto-1.0.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-core-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-recipes-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guava-14.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-core_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-common_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-launcher_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr4-runtime-4.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-util-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jaxb-api-2.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-io-2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-encoding-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpcore-4.4.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-xc-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/protobuf-java-2.5.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-configuration-1.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compress-1.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-mapreduce-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ST4-4.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-core-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-shuffle-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-repl_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/opencsv-2.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-logging-1.1.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-cli-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-client-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-hdfs-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/log4j-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-column-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill_2.11-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stringtemplate-3.2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-common-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-shuffle_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-kvstore_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jta-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javassist-3.18.1-GA.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-httpclient-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jets3t-0.9.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-json-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bcprov-jdk15on-1.58.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/oro-2.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsp-api-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1506645932520_24630151","Timestamp":1524182082734,"User":"edlu"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182111695,"Executor ID":"1","Executor Info":{"Host":"node1404.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stdout?start=-4096","stderr":"http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node1404.grid.company.com","Port":34043},"Maximum Memory":956615884,"Timestamp":1524182111795,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112088,"Executor ID":"3","Executor Info":{"Host":"node0998.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stdout?start=-4096","stderr":"http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"node0998.grid.company.com","Port":45265},"Maximum Memory":956615884,"Timestamp":1524182112208,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112278,"Executor ID":"4","Executor Info":{"Host":"node4243.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stdout?start=-4096","stderr":"http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"node4243.grid.company.com","Port":16084},"Maximum Memory":956615884,"Timestamp":1524182112408,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112471,"Executor ID":"2","Executor Info":{"Host":"node4045.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stdout?start=-4096","stderr":"http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"node4045.grid.company.com","Port":29262},"Maximum Memory":956615884,"Timestamp":1524182112578,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line44.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line44.$read$$iw$$iw$$iw$$iw.(:59)\n$line44.$read$$iw$$iw$$iw.(:61)\n$line44.$read$$iw$$iw.(:63)\n$line44.$read$$iw.(:65)\n$line44.$read.(:67)\n$line44.$read$.(:71)\n$line44.$read$.()\n$line44.$eval$.$print$lzycompute(:7)\n$line44.$eval$.$print(:6)\n$line44.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1524182125829} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1524182125832} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line48.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line48.$read$$iw$$iw$$iw$$iw.(:59)\n$line48.$read$$iw$$iw$$iw.(:61)\n$line48.$read$$iw$$iw.(:63)\n$line48.$read$$iw.(:65)\n$line48.$read.(:67)\n$line48.$read$.(:71)\n$line48.$read$.()\n$line48.$eval$.$print$lzycompute(:7)\n$line48.$eval$.$print(:6)\n$line48.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1524182128463} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1524182128463} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at :40","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- AnalysisBarrier\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nappId: string, attemptId: string, name: string, mode: string, completed: string, duration: string, endTime: string, endTimeEpoch: string, lastUpdated: string, lastUpdatedEpoch: string, sparkUser: string, startTime: string, startTimeEpoch: string, appSparkVersion: string, endDate: string, azkaban.link.workflow.url: string, azkaban.link.execution.url: string, azkaban.link.job.url: string, user.name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@485d3d1, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) LocalLimit 21\n +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@485d3d1, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"LocalLimit","simpleString":"LocalLimit 21","children":[{"nodeName":"Project","simpleString":"Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"InMemoryTableScan","simpleString":"InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]","children":[],"metrics":[{"name":"number of output rows","accumulatorId":35,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":36,"metricType":"timing"}]}],"metrics":[]}],"metrics":[]}],"metrics":[]}],"metrics":[{"name":"duration total (min, med, max)","accumulatorId":34,"metricType":"timing"}]}],"metrics":[]},"time":1524182129952} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1524182130194,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]}],"Stage IDs":[0,1,2],"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1524182130350,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1524182130350,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1524182130350,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182142286,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"154334487","Value":"154334486","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"19666","Value":"19665","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":37809697,"Value":37809697,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":91545212,"Value":91545212,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":20002743,"Value":20002743,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":407,"Value":407,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":1856,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":9020410971,"Value":9020410971,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":11146,"Value":11146,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":574344183,"Value":574344183,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":714,"Value":714,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":714,"Executor Deserialize CPU Time":574344183,"Executor Run Time":11146,"Executor CPU Time":9020410971,"Result Size":1856,"JVM GC Time":407,"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":20002743,"Shuffle Write Time":91545212,"Shuffle Records Written":466636},"Input Metrics":{"Bytes Read":37809697,"Records Read":466636},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1524182130350,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182143009,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"206421303","Value":"360755789","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"20604","Value":"40269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423609,"Value":88233306,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":104125550,"Value":195670762,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":26424033,"Value":46426776,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":374,"Value":781,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":3712,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11039226628,"Value":20059637599,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":11978,"Value":23124,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":526915936,"Value":1101260119,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":622,"Value":1336,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":622,"Executor Deserialize CPU Time":526915936,"Executor Run Time":11978,"Executor CPU Time":11039226628,"Result Size":1856,"JVM GC Time":374,"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":26424033,"Shuffle Write Time":104125550,"Shuffle Records Written":624246},"Input Metrics":{"Bytes Read":50423609,"Records Read":624246},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182143160,"Executor ID":"5","Executor Info":{"Host":"node2477.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stdout?start=-4096","stderr":"http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"node2477.grid.company.com","Port":20123},"Maximum Memory":956615884,"Timestamp":1524182143406,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144246,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"1920974","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"3562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"38","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1813","Value":"1812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"195602","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"3563","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1558","Value":"1557","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":3563,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":36845111,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":27318908,"Value":27318908,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":3562,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":349287,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":33,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1498974375,"Value":1498974375,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1922,"Value":1922,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":49547405,"Value":49547405,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":56,"Executor Deserialize CPU Time":49547405,"Executor Run Time":1922,"Executor CPU Time":1498974375,"Result Size":2394,"JVM GC Time":33,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":27318908,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144444,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"23365","Value":"63634","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":105575962,"Value":301246724,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":326,"Value":1107,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11931694025,"Value":31991331624,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13454,"Value":36578,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":531799977,"Value":1633060096,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":594,"Value":1930,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":594,"Executor Deserialize CPU Time":531799977,"Executor Run Time":13454,"Executor CPU Time":11931694025,"Result Size":1856,"JVM GC Time":326,"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":22950296,"Shuffle Write Time":105575962,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144840,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"24254","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":106963069,"Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":342,"Value":1449,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":12267596062,"Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13858,"Value":50436,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":519573839,"Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":573,"Value":2503,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":573,"Executor Deserialize CPU Time":519573839,"Executor Run Time":13858,"Executor CPU Time":12267596062,"Result Size":1856,"JVM GC Time":342,"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":31362123,"Shuffle Write Time":106963069,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":592412824,"JVM Used Nonheap Memory":202907152,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":905801,"Offheap Storage Memory":0,"Onheap Unified Memory":905801,"Offheap Unified Memory":0,"Direct Memory":355389,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":523121272,"JVM Used Nonheap Memory":88280720,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":52050147,"Offheap Storage Memory":0,"Onheap Unified Memory":52050147,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":214174608,"JVM Used Nonheap Memory":91548704,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47399168,"Offheap Storage Memory":0,"Onheap Unified Memory":47399168,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":518613056,"JVM Used Nonheap Memory":95657456,"Onheap Execution Memory":37748736,"Offheap Execution Memory":0,"Onheap Storage Memory":63104457,"Offheap Storage Memory":0,"Onheap Unified Memory":100853193,"Offheap Unified Memory":0,"Direct Memory":126261,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":726805712,"JVM Used Nonheap Memory":90709624,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69535048,"Offheap Storage Memory":0,"Onheap Unified Memory":69535048,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Completion Time":1524182144852,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":2503,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":50436,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":1449,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145327,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"49","Value":"86","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"2002","Value":"3814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1755","Value":"3312","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":32035583,"Value":59354491,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4788,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1785119941,"Value":3284094316,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":2182,"Value":4104,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":71500541,"Value":121047946,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":136,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":136,"Executor Deserialize CPU Time":71500541,"Executor Run Time":2182,"Executor CPU Time":1785119941,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349006,"Shuffle Write Time":32035583,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145971,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"5212268","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"9572","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"121634812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"9","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1703","Value":"5517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"525948","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"9573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1609","Value":"4921","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":9573,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":97944567,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":20055909,"Value":79410400,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":9572,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":941007,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":121634816,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":7182,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":896878991,"Value":4180973307,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1722,"Value":5826,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2787355,"Value":123835301,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":195,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2787355,"Executor Run Time":1722,"Executor CPU Time":896878991,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":242714,"Shuffle Write Time":20055909,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182147549,"Executor ID":"6","Executor Info":{"Host":"node6644.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stdout?start=-4096","stderr":"http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"node6644.grid.company.com","Port":8445},"Maximum Memory":956615884,"Timestamp":1524182147706,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182149826,"Executor ID":"7","Executor Info":{"Host":"node6340.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096","stderr":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"node6340.grid.company.com","Port":5933},"Maximum Memory":956615884,"Timestamp":1524182149983,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182152418,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"48","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"6093","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"5951","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":49790497,"Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":920,"Value":1015,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9619,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5299274511,"Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":7847,"Value":13673,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687811857,"Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1037,"Value":1232,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1037,"Executor Deserialize CPU Time":687811857,"Executor Run Time":7847,"Executor CPU Time":5299274511,"Result Size":2437,"JVM GC Time":920,"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":355051,"Shuffle Write Time":49790497,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":629553808,"JVM Used Nonheap Memory":205304696,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":905801,"Offheap Storage Memory":0,"Onheap Unified Memory":905801,"Offheap Unified Memory":0,"Direct Memory":397602,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":595946552,"JVM Used Nonheap Memory":91208368,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468944,"Offheap Storage Memory":0,"Onheap Unified Memory":58468944,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":755008624,"JVM Used Nonheap Memory":100519936,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47962185,"Offheap Storage Memory":0,"Onheap Unified Memory":47962185,"Offheap Unified Memory":0,"Direct Memory":98230,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":518613056,"JVM Used Nonheap Memory":95657456,"Onheap Execution Memory":37748736,"Offheap Execution Memory":0,"Onheap Storage Memory":63104457,"Offheap Storage Memory":0,"Onheap Unified Memory":100853193,"Offheap Unified Memory":0,"Direct Memory":126261,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":726805712,"JVM Used Nonheap Memory":90709624,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69535048,"Offheap Storage Memory":0,"Onheap Unified Memory":69535048,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Completion Time":1524182152419,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":1232,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":13673,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":1015,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9619,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1524182152447,"Executor ID":"4","Host":"node4243.grid.company.com","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":8,"Index":0,"Attempt":0,"Launch Time":1524182152447,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1524182153103,"Failed":false,"Killed":false,"Accumulables":[{"ID":34,"Name":"duration total (min, med, max)","Update":"1","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":35,"Name":"number of output rows","Update":"6928","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Update":"452","Value":"451","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"number of output rows","Update":"10945","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"number of output rows","Update":"62","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Update":"33619967","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":22,"Name":"duration total (min, med, max)","Update":"323","Value":"322","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"peak memory total (min, med, max)","Update":"34078719","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Update":"10","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"duration total (min, med, max)","Update":"367","Value":"366","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Update":11007,"Value":11007,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":124513,"Value":124513,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":314162,"Value":314162,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":6,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Update":67698688,"Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Update":4642,"Value":4642,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Update":517655714,"Value":517655714,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.executorRunTime","Update":589,"Value":589,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Update":45797784,"Value":45797784,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Update":50,"Value":50,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":50,"Executor Deserialize CPU Time":45797784,"Executor Run Time":589,"Executor CPU Time":517655714,"Result Size":4642,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":6,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":314162,"Remote Bytes Read To Disk":0,"Local Bytes Read":124513,"Total Records Read":11007},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":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":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Completion Time":1524182153104,"Accumulables":[{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Value":11007,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"number of output rows","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":89,"Name":"internal.metrics.executorRunTime","Value":589,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":6,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"number of output rows","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":91,"Name":"internal.metrics.resultSize","Value":4642,"Internal":true,"Count Failed Values":true},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":314162,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"peak memory total (min, med, max)","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":22,"Name":"duration total (min, med, max)","Value":"322","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Value":45797784,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"duration total (min, med, max)","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Value":"451","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Value":50,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Value":517655714,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":2,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"number of output rows","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":124513,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"duration total (min, med, max)","Value":"366","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1524182153112,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1524182153139} {"Event":"SparkListenerUnpersistRDD","RDD ID":2} {"Event":"SparkListenerUnpersistRDD","RDD ID":20} -{"Event":"SparkListenerApplicationEnd","Timestamp":1521483792501} +{"Event":"SparkListenerApplicationEnd","Timestamp":1524182189134} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 4f8081f2e942..b2d4505eaf87 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -213,7 +213,8 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) - val executorUpdates = new ExecutorMetrics(1234567L, 123456L, 12345L, 1234L, 123L, 12L) + val executorUpdates = new ExecutorMetrics(1234567L, 123456L, 543L, 12345L, 1234L, 123L, 12L, + 432L, 321L, 654L, 765L) val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId, executorUpdates)) if (executorShouldReregister) { 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 59645b2cf181..6b906a88183c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -48,6 +48,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} */ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter with Logging { + import EventLoggingListenerSuite._ private val fileSystem = Utils.getHadoopFileSystem("/", @@ -273,54 +274,95 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus(conf) - // list of events and if they should be logged + // expected ExecutorMetricsUpdate, for the given stage id and executor id + val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] = + Map( + ((0, "1"), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))), + ((0, "2"), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))), + ((1, "1"), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))), + ((1, "2"), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + + // Events to post. val events = Array( - (SparkListenerApplicationStart("executionMetrics", None, - 1L, "update", None), true), - (createExecutorAddedEvent(1), true), - (createExecutorAddedEvent(2), true), - (createStageSubmittedEvent(0), true), - (createExecutorMetricsUpdateEvent(1, 10L, 5000L, 50L, 0L, 0L, 0L), true), // new stage - (createExecutorMetricsUpdateEvent(2, 10L, 3500L, 20L, 0L, 0L, 0L), true), // new stage - (createExecutorMetricsUpdateEvent(1, 15L, 4000L, 50L, 0L, 0L, 0L), false), - (createExecutorMetricsUpdateEvent(2, 15L, 3500L, 10L, 0L, 20L, 0L), true), // onheap storage - (createExecutorMetricsUpdateEvent(1, 20L, 6000L, 50L, 0L, 30L, 0L), true), // JVM used - (createExecutorMetricsUpdateEvent(2, 20L, 3500L, 15L, 0L, 20L, 0L), true), // onheap unified - (createStageSubmittedEvent(1), true), - (createExecutorMetricsUpdateEvent(1, 25L, 3000L, 15L, 0L, 0L, 0L), true), // new stage - (createExecutorMetricsUpdateEvent(2, 25L, 6000L, 50L, 0L, 0L, 0L), true), // new stage - (createStageCompletedEvent(0), true), - (createExecutorMetricsUpdateEvent(1, 30L, 3000L, 20L, 0L, 0L, 0L), true), // onheap execution - (createExecutorMetricsUpdateEvent(2, 30L, 5500L, 20L, 0L, 0L, 0L), false), - (createExecutorMetricsUpdateEvent(1, 35L, 3000L, 5L, 25L, 0L, 0L), true), // offheap execution - (createExecutorMetricsUpdateEvent(2, 35L, 5500L, 25L, 0L, 0L, 30L), true), // offheap storage - (createExecutorMetricsUpdateEvent(1, 40L, 3000L, 8L, 20L, 0L, 0L), false), - (createExecutorMetricsUpdateEvent(2, 40L, 5500L, 25L, 0L, 0L, 30L), false), - (createStageCompletedEvent(1), true), - (SparkListenerApplicationEnd(1000L), true)) + SparkListenerApplicationStart("executionMetrics", None, + 1L, "update", None), + createExecutorAddedEvent(1), + createExecutorAddedEvent(2), + createStageSubmittedEvent(0), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)), + createStageSubmittedEvent(1), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)), + createStageCompletedEvent(0), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)), + createExecutorRemovedEvent(1), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)), + createStageCompletedEvent(1), + SparkListenerApplicationEnd(1000L)) // play the events for the event logger eventLogger.start() listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem])) listenerBus.addToEventLogQueue(eventLogger) - for ((event, included) <- events) { - listenerBus.post(event) - } + events.foreach(event => listenerBus.post(event)) listenerBus.stop() eventLogger.stop() - // Verify the log file contains the expected events + // Verify the log file contains the expected events. + // Posted events should be logged, except for ExecutorMetricsUpdate events -- these + // are consolidated, and the peak values for each stage are logged at stage end. val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) try { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) - assert(lines.size === 19) + assert(lines.size === 14) assert(lines(0).contains("SparkListenerLogStart")) assert(lines(1).contains("SparkListenerApplicationStart")) assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) var i = 1 - for ((event, included) <- events) { - if (included) { + events.foreach {event => + event match { + case metricsUpdate: SparkListenerExecutorMetricsUpdate => + case stageCompleted: SparkListenerStageCompleted => + for (j <- 1 to 2) { + checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId, + expectedMetricsEvents) + i += 1 + } + checkEvent(lines(i), event) + i += 1 + case _ => checkEvent(lines(i), event) i += 1 } @@ -331,55 +373,58 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } /** Create a stage submitted event for the specified stage Id. */ - private def createStageSubmittedEvent(stageId: Int) = + private def createStageSubmittedEvent(stageId: Int) = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "details")) + } /** Create a stage completed event for the specified stage Id. */ - private def createStageCompletedEvent(stageId: Int) = + private def createStageCompletedEvent(stageId: Int) = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "details")) + } /** Create an executor added event for the specified executor Id. */ - private def createExecutorAddedEvent(executorId: Int) = + private def createExecutorAddedEvent(executorId: Int) = { SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty)) + } + + /** Create an executor added event for the specified executor Id. */ + private def createExecutorRemovedEvent(executorId: Int) = { + SparkListenerExecutorRemoved(0L, executorId.toString, "test") + } /** Create an executor metrics update event, with the specified executor metrics values. */ private def createExecutorMetricsUpdateEvent( - executorId: Int, time: Long, - jvmUsedMemory: Long, - onHeapExecutionMemory: Long, - offHeapExecutionMemory: Long, - onHeapStorageMemory: Long, - offHeapStorageMemory: Long): SparkListenerExecutorMetricsUpdate = { + executorId: Int, + executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { val taskMetrics = TaskMetrics.empty taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) - val executorUpdates = new ExecutorMetrics(time, jvmUsedMemory, onHeapExecutionMemory, - offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory) - SparkListenerExecutorMetricsUpdate( executorId.toString, accum, Some(executorUpdates)) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) } /** Check that the two ExecutorMetrics match */ private def checkExecutorMetrics( - executorMetrics1: Option[ExecutorMetrics], - executorMetrics2: Option[ExecutorMetrics]) = { - executorMetrics1 match { - case Some(e1) => - executorMetrics2 match { - case Some(e2) => - assert(e1.timestamp === e2.timestamp) - assert(e1.jvmUsedMemory === e2.jvmUsedMemory) - assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory) - assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory) - assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory) - assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory) - } - case None => + executorMetrics1: Option[ExecutorMetrics], + executorMetrics2: Option[ExecutorMetrics]) = { + (executorMetrics1, executorMetrics2) match { + case (Some(e1), Some(e2)) => + assert(e1.timestamp === e2.timestamp) + assert(e1.jvmUsedHeapMemory === e2.jvmUsedHeapMemory) + assert(e1.jvmUsedNonHeapMemory === e2.jvmUsedNonHeapMemory) + assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory) + assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory) + assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory) + assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory) + assert(e1.onHeapUnifiedMemory === e2.onHeapUnifiedMemory) + assert(e1.offHeapUnifiedMemory === e2.offHeapUnifiedMemory) + assert(e1.directMemory === e2.directMemory) + assert(e1.mappedMemory === e2.mappedMemory) + case (None, None) => + case _ => assert(false) - case None => - assert(executorMetrics2.isEmpty) } } @@ -417,6 +462,32 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } + /** + * Check that the Spark history log line is an ExecutorMetricsUpdate, and matches the expected + * value for the stage and executor. + * + * @param line the Spark history log line + * @param stageId the stage ID the ExecutorMetricsUpdate is associated with + * @param expectedEvents map of expected ExecutorMetricsUpdate events, for (stageId, executorId) + */ + private def checkExecutorMetricsUpdate( + line: String, + stageId: Int, + expectedEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate]): Unit = { + JsonProtocol.sparkEventFromJson(parse(line)) match { + case executorMetrics: SparkListenerExecutorMetricsUpdate => + expectedEvents.get((stageId, executorMetrics.execId)) match { + case Some(expectedMetrics) => + assert(executorMetrics.accumUpdates.isEmpty) + checkExecutorMetrics(executorMetrics.executorUpdates, expectedMetrics.executorUpdates) + case None => + assert(false) + } + case _ => + assertTypeError("expecting SparkListenerExecutorMetricsUpdate") + } + } + private def readLines(in: InputStream): Seq[String] = { Source.fromInputStream(in).getLines().toSeq } 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 1cd71955ad4d..136f20f8db04 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -22,18 +22,19 @@ import java.lang.{Integer => JInteger, Long => JLong} import java.util.{Arrays, Date, Properties} import scala.collection.JavaConverters._ +import scala.collection.immutable.Map import scala.reflect.{classTag, ClassTag} import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.api.v1 +import org.apache.spark.status.api.v1.PeakMemoryMetrics import org.apache.spark.storage._ import org.apache.spark.util.Utils -import org.apache.spark.util.kvstore._ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { @@ -1208,6 +1209,75 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } } + test("executor metrics updates") { + val listener = new AppStatusListener(store, conf, true) + + val driver = BlockManagerId(SparkContext.DRIVER_IDENTIFIER, "localhost", 42) + + listener.onExecutorAdded(createExecutorAddedEvent(1)) + listener.onExecutorAdded(createExecutorAddedEvent(2)) + listener.onStageSubmitted(createStageSubmittedEvent(0)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + listener.onStageSubmitted(createStageSubmittedEvent(1)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(25L, 7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + listener.onStageCompleted(createStageCompletedEvent(0)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + listener.onExecutorRemoved(createExecutorRemovedEvent(1)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + listener.onStageCompleted(createStageCompletedEvent(1)) + + // expected peak values for each executor + val expectedValues = Map( + "1" -> new PeakMemoryMetrics(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), + "2" -> new PeakMemoryMetrics(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) + + // check that the stored peak values match the expected values + for ((id, metrics) <- expectedValues) { + check[ExecutorSummaryWrapper](id) { exec => + assert(exec.info.id === id) + exec.info.peakMemoryMetrics match { + case Some(actual) => + assert(actual.jvmUsedHeapMemory == metrics.jvmUsedHeapMemory) + assert(actual.jvmUsedNonHeapMemory == metrics.jvmUsedNonHeapMemory) + assert(actual.onHeapExecutionMemory == metrics.onHeapExecutionMemory) + assert(actual.offHeapExecutionMemory == metrics.offHeapExecutionMemory) + assert(actual.onHeapStorageMemory == metrics.onHeapStorageMemory) + assert(actual.offHeapStorageMemory == metrics.offHeapStorageMemory) + assert(actual.onHeapUnifiedMemory == metrics.onHeapUnifiedMemory) + assert(actual.offHeapUnifiedMemory == metrics.offHeapUnifiedMemory) + assert(actual.directMemory == metrics.directMemory) + assert(actual.mappedMemory == metrics.mappedMemory) + case _ => + assert(false) + } + } + } + } + private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptNumber) private def check[T: ClassTag](key: Any)(fn: T => Unit): Unit = { @@ -1245,4 +1315,36 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } + /** Create a stage submitted event for the specified stage Id. */ + private def createStageSubmittedEvent(stageId: Int) = { + SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + } + + /** Create a stage completed event for the specified stage Id. */ + private def createStageCompletedEvent(stageId: Int) = { + SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + } + + /** 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)) + } + + /** Create an executor added event for the specified executor Id. */ + private def createExecutorRemovedEvent(executorId: Int) = { + SparkListenerExecutorRemoved(0L, executorId.toString, "test") + } + + /** Create an executor metrics update event, with the specified executor metrics values. */ + private def createExecutorMetricsUpdateEvent( + executorId: Int, + executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { + val taskMetrics = TaskMetrics.empty + taskMetrics.incDiskBytesSpilled(111) + taskMetrics.incMemoryBytesSpilled(222) + val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) + } } 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 e0cc3c46017c..afe11afe3e5c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -94,7 +94,8 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - val executorUpdates = Some(new ExecutorMetrics(1234567L, 123456L, 12345L, 1234L, 123L, 12L)) + val executorUpdates = Some(new ExecutorMetrics(1234567L, 543L, 123456L, 12345L, 1234L, 123L, + 12L, 432L, 321L, 654L, 765L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) } val blockUpdated = @@ -682,21 +683,18 @@ private[spark] object JsonProtocolSuite extends Assertions { } private def assertEquals(metrics1: Option[ExecutorMetrics], metrics2: Option[ExecutorMetrics]) { - metrics1 match { - case Some(m1) => - metrics2 match { - case Some(m2) => - assert(m1.timestamp === m2.timestamp) - assert(m1.jvmUsedMemory === m2.jvmUsedMemory) - assert(m1.onHeapExecutionMemory === m2.onHeapExecutionMemory) - assert(m1.offHeapExecutionMemory === m2.offHeapExecutionMemory) - assert(m1.onHeapStorageMemory === m2.onHeapStorageMemory) - assert(m1.offHeapStorageMemory === m2.offHeapStorageMemory) - case None => - assert(false) - } - case None => - assert(metrics2.isEmpty) + (metrics1, metrics2) match { + case (Some(m1), Some(m2)) => + assert(m1.timestamp === m2.timestamp) + assert(m1.jvmUsedHeapMemory === m2.jvmUsedHeapMemory) + assert(m1.jvmUsedNonHeapMemory === m2.jvmUsedNonHeapMemory) + assert(m1.onHeapExecutionMemory === m2.onHeapExecutionMemory) + assert(m1.offHeapExecutionMemory === m2.offHeapExecutionMemory) + assert(m1.onHeapStorageMemory === m2.onHeapStorageMemory) + assert(m1.offHeapStorageMemory === m2.offHeapStorageMemory) + case (None, None) => + case _ => + assert(false) } } @@ -867,9 +865,10 @@ private[spark] object JsonProtocolSuite extends Assertions { internal, countFailedValues, metadata) /** Creates an SparkListenerExecutorMetricsUpdate event */ - private def makeExecutorMetricsUpdate(execId: String, - includeTaskMetrics: Boolean, - includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { + private def makeExecutorMetricsUpdate( + execId: String, + includeTaskMetrics: Boolean, + includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { val taskMetrics = if (includeTaskMetrics) { Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), @@ -879,7 +878,7 @@ private[spark] object JsonProtocolSuite extends Assertions { } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(1234567L, 123456L, 0L, 0L, 0L, 0L)) + Some(new ExecutorMetrics(1234567L, 123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L)) } else { None } @@ -2076,11 +2075,16 @@ private[spark] object JsonProtocolSuite extends Assertions { | ], | "Executor Metrics Updated" : { | "Timestamp" : 1234567, - | "JVM Used Memory" : 123456, + | "JVM Used Heap Memory" : 543, + | "JVM Used Nonheap Memory" : 123456, | "Onheap Execution Memory" : 12345, | "Offheap Execution Memory" : 1234, | "Onheap Storage Memory" : 123, - | "Offheap Storage Memory" : 12 + | "Offheap Storage Memory" : 12, + | "Onheap Unified Memory" : 432, + | "Offheap Unified Memory" : 321, + | "Direct Memory" : 654, + | "Mapped Memory" : 765 | } | |} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 8ee7822282e4..8521d2a4c7db 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -73,7 +73,7 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.InternalNode"), ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.Node"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.this") + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.this"), // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"), From 10ed328bfcf160711e7619aac23472f97bf1c976 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Mon, 14 May 2018 17:24:22 -0700 Subject: [PATCH 04/29] Add configuration parameter spark.eventLog.logExecutorMetricsUpdates.enabled to enable/disable executor metrics update logging. Code review comments. --- .../scala/org/apache/spark/Heartbeater.scala | 5 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/internal/config/package.scala | 5 ++ .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scheduler/EventLoggingListener.scala | 66 +++++++++++-------- 5 files changed, 50 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index b36285ed42e6..2c89cc14189d 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -26,11 +26,12 @@ import org.apache.spark.util.{ThreadUtils, Utils} * intervals of intervalMs. * * @param reportHeartbeat the heartbeat reporting function to call. + * @param name the thread name for the heartbeater. * @param intervalMs the interval between heartbeats. */ -private[spark] class Heartbeater(reportHeartbeat: () => Unit, intervalMs: Long) { +private[spark] class Heartbeater(reportHeartbeat: () => Unit, name: String, intervalMs: Long) { // Executor for the heartbeat task - private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") + private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor(name) /** Schedules a task to report a heartbeat. */ private[spark] def start(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bbc5f375bc9f..bdf2553ce6e9 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -155,7 +155,7 @@ private[spark] class Executor( private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] // Executor for the heartbeat task. - private val heartbeater = new Heartbeater(reportHeartBeat, + private val heartbeater = new Heartbeater(reportHeartBeat, "executor-heartbeater", conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) // must be initialized before running startDriverHeartbeat() diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 6bb98c37b447..a5366aa7de42 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -69,6 +69,11 @@ package object config { .bytesConf(ByteUnit.KiB) .createWithDefaultString("100k") + private[spark] val EVENT_LOG_EXECUTOR_METRICS_UPDATES = + ConfigBuilder("spark.eventLog.logExecutorMetricsUpdates.enabled") + .booleanConf + .createWithDefault(true) + private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5b628929ed3d..fd950f0c207f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -211,7 +211,7 @@ class DAGScheduler( taskScheduler.setDAGScheduler(this) /** driver heartbeat for collecting metrics */ - private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, + private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, "driver-heartbeater", sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) /** BufferPoolMXBean for direct memory */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 5fae14bb1708..a795186f20b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -52,6 +52,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} * spark.eventLog.overwrite - Whether to overwrite any existing files. * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams + * spark.eventLog.logExecutorMetricsUpdates.enabled - Whether to log executor metrics updates */ private[spark] class EventLoggingListener( appId: String, @@ -70,6 +71,7 @@ private[spark] class EventLoggingListener( private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) + private val shouldLogExecutorMetricsUpdates = sparkConf.get(EVENT_LOG_EXECUTOR_METRICS_UPDATES) private val testing = sparkConf.get(EVENT_LOG_TESTING) private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) @@ -82,7 +84,7 @@ private[spark] class EventLoggingListener( private val compressionCodecName = compressionCodec.map { c => CompressionCodec.getShortName(c.getClass.getName) } - +logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecutorMetricsUpdates) // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None @@ -162,9 +164,11 @@ private[spark] class EventLoggingListener( // Events that do not trigger a flush override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { logEvent(event) - // clear the peak metrics when a new stage starts - liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), - new mutable.HashMap[String, PeakExecutorMetrics]()) + if (shouldLogExecutorMetricsUpdates) { + // record the peak metrics for the new stage + liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), + new mutable.HashMap[String, PeakExecutorMetrics]()) + } } override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) @@ -179,22 +183,30 @@ private[spark] class EventLoggingListener( // Events that trigger a flush override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { - // log the peak executor metrics for the stage, for each executor - val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() - val executorMap = liveStageExecutorMetrics.remove( - (event.stageInfo.stageId, event.stageInfo.attemptNumber())) - executorMap.foreach { - executorEntry => { - for ((executorId, peakExecutorMetrics) <- executorEntry) { - val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory, - peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory, - peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory, - peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory, - peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory, - peakExecutorMetrics.mappedMemory) - val executorUpdate = new SparkListenerExecutorMetricsUpdate( - executorId, accumUpdates, Some(executorMetrics)) - logEvent(executorUpdate) + if (shouldLogExecutorMetricsUpdates) { + // clear out any previous attempts, that did not have a stage completed event + val prevAttemptId = event.stageInfo.attemptNumber() - 1 + for (attemptId <- 0 to prevAttemptId) { + liveStageExecutorMetrics.remove((event.stageInfo.stageId, attemptId)) + } + + // log the peak executor metrics for the stage, for each executor + val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() + val executorMap = liveStageExecutorMetrics.remove( + (event.stageInfo.stageId, event.stageInfo.attemptNumber())) + executorMap.foreach { + executorEntry => { + for ((executorId, peakExecutorMetrics) <- executorEntry) { + val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory, + peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory, + peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory, + peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory, + peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory, + peakExecutorMetrics.mappedMemory) + val executorUpdate = new SparkListenerExecutorMetricsUpdate( + executorId, accumUpdates, Some(executorMetrics)) + logEvent(executorUpdate) + } } } } @@ -266,12 +278,14 @@ private[spark] class EventLoggingListener( } override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - // For the active stages, record any new peak values for the memory metrics for the executor - event.executorUpdates.foreach { executorUpdates => - liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => - val peakMetrics = peakExecutorMetrics.getOrElseUpdate( - event.execId, new PeakExecutorMetrics()) - peakMetrics.compareAndUpdate(executorUpdates) + if (shouldLogExecutorMetricsUpdates) { + // For the active stages, record any new peak values for the memory metrics for the executor + event.executorUpdates.foreach { executorUpdates => + liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => + val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + event.execId, new PeakExecutorMetrics()) + peakMetrics.compareAndUpdate(executorUpdates) + } } } } From 2d2036760a298c7434eb4816c1bf045c43713e6f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 23 May 2018 14:37:26 -0500 Subject: [PATCH 05/29] wip on enum based metrics --- .../apache/spark/scheduler/MemoryTypes.java | 67 +++++++++++++ .../spark/scheduler/PeakExecutorMetrics.scala | 93 ++++--------------- .../org/apache/spark/status/api/v1/api.scala | 17 +--- 3 files changed, 90 insertions(+), 87 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java diff --git a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java new file mode 100644 index 000000000000..876233916f0b --- /dev/null +++ b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java @@ -0,0 +1,67 @@ +/* + * 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.scheduler; + +import org.apache.spark.executor.ExecutorMetrics; + +public enum MemoryTypes { + JvmUsedMemory{ + @Override + long get(ExecutorMetrics em) { + return em.jvmUsedMemory(); + } + }, + OnHeapExecutionMemory { + @Override + long get(ExecutorMetrics em) { + return em.onHeapExecutionMemory(); + } + }, + OffHeapExecutionMemory { + @Override + long get(ExecutorMetrics em) { + return em.offHeapExecutionMemory(); + } + }, + OnHeapStorageMemory { + @Override + long get(ExecutorMetrics em) { + return em.onHeapStorageMemory(); + } + }, + OffHeapStorageMemory { + @Override + long get(ExecutorMetrics em) { + return em.offHeapStorageMemory(); + } + }, + OnHeapUnifiedMemory { + @Override + long get(ExecutorMetrics em) { + return em.onHeapExecutionMemory() + em.onHeapStorageMemory(); + } + }, + OffHeapUnifiedMemory { + @Override + long get(ExecutorMetrics em) { + return em.offHeapExecutionMemory() + em.offHeapStorageMemory(); + } + }; + + abstract long get(ExecutorMetrics em); +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index d554c938f5e1..a91aabde9306 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -25,36 +25,8 @@ import org.apache.spark.status.api.v1.PeakMemoryMetrics * values have been recorded yet. */ private[spark] class PeakExecutorMetrics { - private var _jvmUsedHeapMemory = -1L; - private var _jvmUsedNonHeapMemory = 0L; - private var _onHeapExecutionMemory = 0L - private var _offHeapExecutionMemory = 0L - private var _onHeapStorageMemory = 0L - private var _offHeapStorageMemory = 0L - private var _onHeapUnifiedMemory = 0L - private var _offHeapUnifiedMemory = 0L - private var _directMemory = 0L - private var _mappedMemory = 0L - - def jvmUsedHeapMemory: Long = _jvmUsedHeapMemory - - def jvmUsedNonHeapMemory: Long = _jvmUsedNonHeapMemory - - def onHeapExecutionMemory: Long = _onHeapExecutionMemory - - def offHeapExecutionMemory: Long = _offHeapExecutionMemory - - def onHeapStorageMemory: Long = _onHeapStorageMemory - - def offHeapStorageMemory: Long = _offHeapStorageMemory - - def onHeapUnifiedMemory: Long = _onHeapUnifiedMemory - - def offHeapUnifiedMemory: Long = _offHeapUnifiedMemory - - def directMemory: Long = _directMemory - - def mappedMemory: Long = _mappedMemory + val metrics = new Array[Long](MemoryTypes.values().length) + metrics(0) = -1 /** * Compare the specified memory values with the saved peak executor memory @@ -66,47 +38,13 @@ private[spark] class PeakExecutorMetrics { def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { var updated: Boolean = false - if (executorMetrics.jvmUsedHeapMemory > _jvmUsedHeapMemory) { - _jvmUsedHeapMemory = executorMetrics.jvmUsedHeapMemory - updated = true - } - if (executorMetrics.jvmUsedNonHeapMemory > _jvmUsedNonHeapMemory) { - _jvmUsedNonHeapMemory = executorMetrics.jvmUsedNonHeapMemory - updated = true + (0 until MemoryTypes.values().length).foreach { metricIdx => + val metricVal = MemoryTypes.values()(metricIdx).get(executorMetrics) + if (metricVal > metrics(metricIdx)) { + updated = true + metrics(metricIdx) = metricVal + } } - if (executorMetrics.onHeapExecutionMemory > _onHeapExecutionMemory) { - _onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory - updated = true - } - if (executorMetrics.offHeapExecutionMemory > _offHeapExecutionMemory) { - _offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory - updated = true - } - if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) { - _onHeapStorageMemory = executorMetrics.onHeapStorageMemory - updated = true - } - if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) { - _offHeapStorageMemory = executorMetrics.offHeapStorageMemory - updated = true - } - if (executorMetrics.onHeapUnifiedMemory > _onHeapUnifiedMemory) { - _onHeapUnifiedMemory = executorMetrics.onHeapUnifiedMemory - updated = true - } - if (executorMetrics.offHeapUnifiedMemory > _offHeapUnifiedMemory) { - _offHeapUnifiedMemory = executorMetrics.offHeapUnifiedMemory - updated = true - } - if (executorMetrics.directMemory > _directMemory) { - _directMemory = executorMetrics.directMemory - updated = true - } - if (executorMetrics.mappedMemory > _mappedMemory) { - _mappedMemory = executorMetrics.mappedMemory - updated = true - } - updated } @@ -115,13 +53,18 @@ private[spark] class PeakExecutorMetrics { * values set. */ def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = { - if (_jvmUsedHeapMemory < 0) { + if (metrics(0) < 0) { None } else { - Some(new PeakMemoryMetrics(_jvmUsedHeapMemory, _jvmUsedNonHeapMemory, - _onHeapExecutionMemory, _offHeapExecutionMemory, _onHeapStorageMemory, - _offHeapStorageMemory, _onHeapUnifiedMemory, _offHeapUnifiedMemory, - _directMemory, _mappedMemory)) + val copy = new PeakMemoryMetrics + System.arraycopy(this.metrics, 0, copy.metrics, 0, this.metrics.length) + Some(copy) } } + + /** Clears/resets the saved peak values. */ + def reset(): Unit = { + (0 until metrics.length).foreach { idx => metrics(idx) = 0} + metrics(0) = -1 + } } 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 15f83213bb6e..aff0dcf7c9be 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 @@ -25,7 +25,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.JobExecutionStatus -import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.scheduler.MemoryTypes case class ApplicationInfo private[spark]( id: String, @@ -108,17 +108,10 @@ class MemoryMetrics private[spark]( val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) -class PeakMemoryMetrics private[spark]( - val jvmUsedHeapMemory: Long, - val jvmUsedNonHeapMemory: Long, - val onHeapExecutionMemory: Long, - val offHeapExecutionMemory: Long, - val onHeapStorageMemory: Long, - val offHeapStorageMemory: Long, - val onHeapUnifiedMemory: Long, - val offHeapUnifiedMemory: Long, - val directMemory: Long, - val mappedMemory: Long) +class PeakMemoryMetrics private[spark]() { + // TODO special json-ification + val metrics = new Array[Long](MemoryTypes.values().length) +} class JobData private[spark]( val jobId: Int, From f904f1e0bc3fab90db7f7aa7cfcf71b9fb26e890 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 23 May 2018 15:50:26 -0500 Subject: [PATCH 06/29] wip ... has both enum and non-enum version --- .../apache/spark/scheduler/MemoryTypes.java | 51 +++--------- .../org/apache/spark/executor/Executor.scala | 55 +++---------- .../spark/executor/ExecutorMetrics.scala | 26 +----- .../apache/spark/metrics/MetricGetter.scala | 80 +++++++++++++++++++ .../apache/spark/scheduler/DAGScheduler.scala | 9 +-- .../scheduler/EventLoggingListener.scala | 9 +-- .../spark/scheduler/PeakExecutorMetrics.scala | 6 +- 7 files changed, 112 insertions(+), 124 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala diff --git a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java index 876233916f0b..9897999d41b1 100644 --- a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java +++ b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java @@ -18,50 +18,21 @@ package org.apache.spark.scheduler; import org.apache.spark.executor.ExecutorMetrics; +import org.apache.spark.memory.MemoryManager; public enum MemoryTypes { - JvmUsedMemory{ + JvmUsedMemory { @Override - long get(ExecutorMetrics em) { - return em.jvmUsedMemory(); + long getValue(MemoryManager memoryManager) { + return 0; } }, - OnHeapExecutionMemory { - @Override - long get(ExecutorMetrics em) { - return em.onHeapExecutionMemory(); - } - }, - OffHeapExecutionMemory { - @Override - long get(ExecutorMetrics em) { - return em.offHeapExecutionMemory(); - } - }, - OnHeapStorageMemory { - @Override - long get(ExecutorMetrics em) { - return em.onHeapStorageMemory(); - } - }, - OffHeapStorageMemory { - @Override - long get(ExecutorMetrics em) { - return em.offHeapStorageMemory(); - } - }, - OnHeapUnifiedMemory { - @Override - long get(ExecutorMetrics em) { - return em.onHeapExecutionMemory() + em.onHeapStorageMemory(); - } - }, - OffHeapUnifiedMemory { - @Override - long get(ExecutorMetrics em) { - return em.offHeapExecutionMemory() + em.offHeapStorageMemory(); - } - }; + OnHeapExecutionMemory, + OffHeapExecutionMemory, + OnHeapStorageMemory, + OffHeapStorageMemory, + OnHeapUnifiedMemory, + OffHeapUnifiedMemory; - abstract long get(ExecutorMetrics em); + abstract long getValue(MemoryManager memoryManager); } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bdf2553ce6e9..43ab51482a91 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -19,13 +19,12 @@ package org.apache.spark.executor import java.io.{File, NotSerializableException} import java.lang.Thread.UncaughtExceptionHandler -import java.lang.management.{BufferPoolMXBean, ManagementFactory} +import java.lang.management.ManagementFactory import java.net.{URI, URL} import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent._ import javax.annotation.concurrent.GuardedBy -import javax.management.ObjectName import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -38,8 +37,9 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.memory.{MemoryManager, SparkOutOfMemoryError, TaskMemoryManager} +import org.apache.spark.metrics.MetricGetter import org.apache.spark.rpc.RpcTimeout -import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription} +import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ @@ -72,12 +72,6 @@ private[spark] class Executor( private val conf = env.conf - // BufferPoolMXBean for direct memory - private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME) - - // BufferPoolMXBean for mapped memory - private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME) - // No ip or host:port - just hostname Utils.checkHost(executorHostname) // must not have port specified. @@ -780,8 +774,7 @@ private[spark] class Executor( val curGCTime = computeTotalGcTime() // get executor level memory metrics - val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager, - directBufferPool, mappedBufferPool) + val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager) for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { @@ -820,42 +813,14 @@ private[spark] object Executor { // used instead. val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties] - val DIRECT_BUFFER_POOL_NAME = "direct" - val MAPPED_BUFFER_POOL_NAME = "mapped" - - /** Get the BufferPoolMXBean for the specified buffer pool. */ - def getBufferPool(pool: String): BufferPoolMXBean = { - val name = new ObjectName("java.nio:type=BufferPool,name=" + pool) - ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, - name.toString, classOf[BufferPoolMXBean]) - } - /** * Get the current executor level memory metrics. - * - * @param memoryManager the memory manager - * @param direct the direct memory buffer pool - * @param mapped the mapped memory buffer pool - * @return the executor memory metrics */ - def getCurrentExecutorMetrics( - memoryManager: MemoryManager, - direct: BufferPoolMXBean, - mapped: BufferPoolMXBean) : ExecutorMetrics = { - val onHeapExecutionMemoryUsed = memoryManager.onHeapExecutionMemoryUsed - val offHeapExecutionMemoryUsed = memoryManager.offHeapExecutionMemoryUsed - val onHeapStorageMemoryUsed = memoryManager.onHeapStorageMemoryUsed - val offHeapStorageMemoryUsed = memoryManager.offHeapStorageMemoryUsed - new ExecutorMetrics(System.currentTimeMillis(), - ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), - ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed(), - onHeapExecutionMemoryUsed, - offHeapExecutionMemoryUsed, - onHeapStorageMemoryUsed, - offHeapStorageMemoryUsed, - onHeapExecutionMemoryUsed + onHeapStorageMemoryUsed, // on heap unified memory - offHeapExecutionMemoryUsed + offHeapStorageMemoryUsed, // off heap unified memory - direct.getMemoryUsed, - mapped.getMemoryUsed) + def getCurrentExecutorMetrics(memoryManager: MemoryManager): ExecutorMetrics = { + val metrics = new ExecutorMetrics(System.currentTimeMillis()) + MetricGetter.idxAndValues.foreach { case (idx, metric) => + metrics.metrics(idx) = metric.getMetricValue(memoryManager) + } + metrics } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 6d35a1c682e2..ce9ad66bafb9 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -18,6 +18,7 @@ package org.apache.spark.executor import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.scheduler.MemoryTypes /** * :: DeveloperApi :: @@ -28,27 +29,8 @@ import org.apache.spark.annotation.DeveloperApi * * @param timestamp the time the metrics were collected, or -1 for Spark history * log events which are logged when a stage has completed - * @param jvmUsedHeapMemory the amount of JVM used heap memory for the executor - * @param jvmUsedNonHeapMemory the amount of JVM used non-heap memory for the executor - * @param onHeapExecutionMemory the amount of on heap execution memory used - * @param offHeapExecutionMemory the amount of off heap execution memory used - * @param onHeapStorageMemory the amount of on heap storage memory used - * @param offHeapStorageMemory the amount of off heap storage memory used - * @param onHeapUnifiedMemory the amount of on heap unified region memory used - * @param offHeapUnifiedMemory the amount of off heap unified region memory used - * @param directMemory the amount of direct memory used - * @param mappedMemory the amount of mapped memory used */ @DeveloperApi -class ExecutorMetrics private[spark] ( - val timestamp: Long, - val jvmUsedHeapMemory: Long, - val jvmUsedNonHeapMemory: Long, - val onHeapExecutionMemory: Long, - val offHeapExecutionMemory: Long, - val onHeapStorageMemory: Long, - val offHeapStorageMemory: Long, - val onHeapUnifiedMemory: Long, - val offHeapUnifiedMemory: Long, - val directMemory: Long, - val mappedMemory: Long) extends Serializable +class ExecutorMetrics private[spark] (val timestamp: Long) extends Serializable { + val metrics = new Array[Long](MemoryTypes.values().length) +} diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala new file mode 100644 index 000000000000..d32ea9df77f7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala @@ -0,0 +1,80 @@ +/* + * 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.metrics + +import java.lang.management.{BufferPoolMXBean, ManagementFactory} +import javax.management.ObjectName + +import org.apache.spark.memory.MemoryManager + +sealed trait MetricGetter { + def getMetricValue(memoryManager: MemoryManager): Long +} + +abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter { + override def getMetricValue(memoryManager: MemoryManager): Long = { + f(memoryManager) + } +} + +abstract class MBeanMetricGetter(mBeanName: String) extends MetricGetter { + private val name = new ObjectName(mBeanName) + val bean = ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, + name.toString, classOf[BufferPoolMXBean]) + + override def getMetricValue(memoryManager: MemoryManager): Long = { + bean.getMemoryUsed + } +} + +case object JVMHeapMemory extends MetricGetter { + override def getMetricValue(memoryManager: MemoryManager): Long = { + ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed() + } +} + +case object JVMOffHeapMemory extends MetricGetter { + override def getMetricValue(memoryManager: MemoryManager): Long = { + ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed() + } +} + +case object OnHeapExecution extends MemoryManagerMetricGetter(_.onHeapExecutionMemoryUsed) + +case object OffHeapExecution extends MemoryManagerMetricGetter(_.offHeapExecutionMemoryUsed) + +case object OnHeapStorage extends MemoryManagerMetricGetter(_.onHeapStorageMemoryUsed) + +case object OffHeapStorage extends MemoryManagerMetricGetter(_.offHeapStorageMemoryUsed) + +case object DirectPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=direct") +case object MappedPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=mapped") + +object MetricGetter { + val values = IndexedSeq( + JVMHeapMemory, + JVMOffHeapMemory, + OnHeapExecution, + OffHeapExecution, + OnHeapStorage, + OffHeapStorage, + DirectPoolMemory, + MappedPoolMemory + ) + + val idxAndValues = values.zipWithIndex.map(_.swap) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index fd950f0c207f..98ed6c75d7eb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -214,12 +214,6 @@ class DAGScheduler( private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, "driver-heartbeater", sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) - /** BufferPoolMXBean for direct memory */ - private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME) - - /** BufferPoolMXBean for mapped memory */ - private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME) - /** * Called by the TaskSetManager to report task's starting. */ @@ -1772,8 +1766,7 @@ class DAGScheduler( /** Reports heartbeat metrics for the driver. */ private def reportHeartBeat(): Unit = { // get driver memory metrics - val driverUpdates = Executor.getCurrentExecutorMetrics( - sc.env.memoryManager, directBufferPool, mappedBufferPool) + val driverUpdates = Executor.getCurrentExecutorMetrics(sc.env.memoryManager) val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, Some(driverUpdates))) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index a795186f20b2..baf7127a8392 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -197,12 +197,9 @@ logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecut executorMap.foreach { executorEntry => { for ((executorId, peakExecutorMetrics) <- executorEntry) { - val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory, - peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory, - peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory, - peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory, - peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory, - peakExecutorMetrics.mappedMemory) + val executorMetrics = new ExecutorMetrics(-1) + System.arraycopy(peakExecutorMetrics.metrics, 0, executorMetrics.metrics, 0, + peakExecutorMetrics.metrics.size) val executorUpdate = new SparkListenerExecutorMetricsUpdate( executorId, accumUpdates, Some(executorMetrics)) logEvent(executorUpdate) diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index a91aabde9306..e40474eb4709 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -39,10 +39,10 @@ private[spark] class PeakExecutorMetrics { var updated: Boolean = false (0 until MemoryTypes.values().length).foreach { metricIdx => - val metricVal = MemoryTypes.values()(metricIdx).get(executorMetrics) - if (metricVal > metrics(metricIdx)) { + val newVal = executorMetrics.metrics(metricIdx) + if ( newVal > metrics(metricIdx)) { updated = true - metrics(metricIdx) = metricVal + metrics(metricIdx) = newVal } } updated From c502ec4c7f55083356187c2906d24440d0168d2f Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 23 May 2018 16:23:44 -0500 Subject: [PATCH 07/29] case objects, mostly complete --- .../apache/spark/scheduler/MemoryTypes.java | 38 ------------------- .../spark/executor/ExecutorMetrics.scala | 4 +- .../apache/spark/metrics/MetricGetter.scala | 4 +- .../spark/scheduler/PeakExecutorMetrics.scala | 5 ++- .../org/apache/spark/status/api/v1/api.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 38 +++++++------------ .../scheduler/EventLoggingListenerSuite.scala | 15 ++------ .../apache/spark/util/JsonProtocolSuite.scala | 10 ++--- 8 files changed, 30 insertions(+), 88 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java diff --git a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java deleted file mode 100644 index 9897999d41b1..000000000000 --- a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java +++ /dev/null @@ -1,38 +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.scheduler; - -import org.apache.spark.executor.ExecutorMetrics; -import org.apache.spark.memory.MemoryManager; - -public enum MemoryTypes { - JvmUsedMemory { - @Override - long getValue(MemoryManager memoryManager) { - return 0; - } - }, - OnHeapExecutionMemory, - OffHeapExecutionMemory, - OnHeapStorageMemory, - OffHeapStorageMemory, - OnHeapUnifiedMemory, - OffHeapUnifiedMemory; - - abstract long getValue(MemoryManager memoryManager); -} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index ce9ad66bafb9..663aecfbebd3 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -18,7 +18,7 @@ package org.apache.spark.executor import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.scheduler.MemoryTypes +import org.apache.spark.metrics.MetricGetter /** * :: DeveloperApi :: @@ -32,5 +32,5 @@ import org.apache.spark.scheduler.MemoryTypes */ @DeveloperApi class ExecutorMetrics private[spark] (val timestamp: Long) extends Serializable { - val metrics = new Array[Long](MemoryTypes.values().length) + val metrics = new Array[Long](MetricGetter.values.length) } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala index d32ea9df77f7..53c87a1261e1 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala @@ -23,6 +23,7 @@ import org.apache.spark.memory.MemoryManager sealed trait MetricGetter { def getMetricValue(memoryManager: MemoryManager): Long + val name = getClass().getName().stripSuffix("$") } abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter { @@ -32,9 +33,8 @@ abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends Metri } abstract class MBeanMetricGetter(mBeanName: String) extends MetricGetter { - private val name = new ObjectName(mBeanName) val bean = ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, - name.toString, classOf[BufferPoolMXBean]) + new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean]) override def getMetricValue(memoryManager: MemoryManager): Long = { bean.getMemoryUsed diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index e40474eb4709..83bb39cf3c3e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.metrics.MetricGetter import org.apache.spark.status.api.v1.PeakMemoryMetrics /** @@ -25,7 +26,7 @@ import org.apache.spark.status.api.v1.PeakMemoryMetrics * values have been recorded yet. */ private[spark] class PeakExecutorMetrics { - val metrics = new Array[Long](MemoryTypes.values().length) + val metrics = new Array[Long](MetricGetter.values.length) metrics(0) = -1 /** @@ -38,7 +39,7 @@ private[spark] class PeakExecutorMetrics { def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { var updated: Boolean = false - (0 until MemoryTypes.values().length).foreach { metricIdx => + (0 until MetricGetter.values.length).foreach { metricIdx => val newVal = executorMetrics.metrics(metricIdx) if ( newVal > metrics(metricIdx)) { updated = true 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 aff0dcf7c9be..3d6360383683 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 @@ -25,7 +25,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.JobExecutionStatus -import org.apache.spark.scheduler.MemoryTypes +import org.apache.spark.metrics.MetricGetter case class ApplicationInfo private[spark]( id: String, @@ -110,7 +110,7 @@ class MemoryMetrics private[spark]( class PeakMemoryMetrics private[spark]() { // TODO special json-ification - val metrics = new Array[Long](MemoryTypes.values().length) + val metrics = new Array[Long](MetricGetter.values.length) } class JobData private[spark]( 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 35b6aeb8f0e7..37b8099f4eb4 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -31,6 +31,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark._ import org.apache.spark.executor._ +import org.apache.spark.metrics.MetricGetter import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -389,17 +390,12 @@ private[spark] object JsonProtocol { * @return the JSON representation */ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { - ("Timestamp" -> executorMetrics.timestamp) ~ - ("JVM Used Heap Memory" -> executorMetrics.jvmUsedHeapMemory) ~ - ("JVM Used Nonheap Memory" -> executorMetrics.jvmUsedNonHeapMemory) ~ - ("Onheap Execution Memory" -> executorMetrics.onHeapExecutionMemory) ~ - ("Offheap Execution Memory" -> executorMetrics.offHeapExecutionMemory) ~ - ("Onheap Storage Memory" -> executorMetrics.onHeapStorageMemory) ~ - ("Offheap Storage Memory" -> executorMetrics.offHeapStorageMemory) ~ - ("Onheap Unified Memory" -> executorMetrics.onHeapUnifiedMemory) ~ - ("Offheap Unified Memory" -> executorMetrics.offHeapUnifiedMemory) ~ - ("Direct Memory" -> executorMetrics.directMemory) ~ - ("Mapped Memory" -> executorMetrics.mappedMemory) + val metrics = MetricGetter.idxAndValues.map { case (idx, metric) => + JField(metric.name, executorMetrics.metrics(idx)) + } + JObject( + (Seq(JField("Timestamp", executorMetrics.timestamp)) ++ metrics): _* + ) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -614,20 +610,12 @@ private[spark] object JsonProtocol { */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val timeStamp = (json \ "Timestamp").extract[Long] - val jvmUsedHeapMemory = (json \ "JVM Used Heap Memory").extract[Long] - val jvmUsedNonHeapMemory = (json \ "JVM Used Nonheap Memory").extract[Long] - val onHeapExecutionMemory = (json \ "Onheap Execution Memory").extract[Long] - val offHeapExecutionMemory = (json \ "Offheap Execution Memory").extract[Long] - val onHeapStorageMemory = (json \ "Onheap Storage Memory").extract[Long] - val offHeapStorageMemory = (json \ "Offheap Storage Memory").extract[Long] - val onHeapUnifiedMemory = (json \ "Onheap Unified Memory").extract[Long] - val offHeapUnifiedMemory = (json \ "Offheap Unified Memory").extract[Long] - val directMemory = (json \ "Direct Memory").extract[Long] - val mappedMemory = (json \ "Mapped Memory").extract[Long] - new ExecutorMetrics(timeStamp, jvmUsedHeapMemory, jvmUsedNonHeapMemory, - onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, - offHeapStorageMemory, onHeapUnifiedMemory, offHeapUnifiedMemory, directMemory, - mappedMemory) + val metrics = new ExecutorMetrics(timeStamp) + MetricGetter.idxAndValues.foreach { case (idx, metric) => + val metricValue = (json \ metric.name).extract[Long] + metrics.metrics(idx) = metricValue + } + metrics } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { 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 6b906a88183c..f6ad8a6f16d8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.io._ -import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.metrics.{MetricGetter, MetricsSystem} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} @@ -412,16 +412,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit (executorMetrics1, executorMetrics2) match { case (Some(e1), Some(e2)) => assert(e1.timestamp === e2.timestamp) - assert(e1.jvmUsedHeapMemory === e2.jvmUsedHeapMemory) - assert(e1.jvmUsedNonHeapMemory === e2.jvmUsedNonHeapMemory) - assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory) - assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory) - assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory) - assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory) - assert(e1.onHeapUnifiedMemory === e2.onHeapUnifiedMemory) - assert(e1.offHeapUnifiedMemory === e2.offHeapUnifiedMemory) - assert(e1.directMemory === e2.directMemory) - assert(e1.mappedMemory === e2.mappedMemory) + (0 until MetricGetter.values.length).foreach { idx => + assert(e1.metrics(idx) === e2.metrics(idx)) + } case (None, None) => case _ => assert(false) 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 afe11afe3e5c..4f3b412dbd43 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -30,6 +30,7 @@ import org.scalatest.exceptions.TestFailedException import org.apache.spark._ import org.apache.spark.executor._ +import org.apache.spark.metrics.MetricGetter import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -686,12 +687,9 @@ private[spark] object JsonProtocolSuite extends Assertions { (metrics1, metrics2) match { case (Some(m1), Some(m2)) => assert(m1.timestamp === m2.timestamp) - assert(m1.jvmUsedHeapMemory === m2.jvmUsedHeapMemory) - assert(m1.jvmUsedNonHeapMemory === m2.jvmUsedNonHeapMemory) - assert(m1.onHeapExecutionMemory === m2.onHeapExecutionMemory) - assert(m1.offHeapExecutionMemory === m2.offHeapExecutionMemory) - assert(m1.onHeapStorageMemory === m2.onHeapStorageMemory) - assert(m1.offHeapStorageMemory === m2.offHeapStorageMemory) + (0 until MetricGetter.values.length).foreach { idx => + assert(m1.metrics(idx) === m2.metrics(idx)) + } case (None, None) => case _ => assert(false) From 2662f6f9c6a7c34cea34b748f6735eb1625b73cb Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Sun, 10 Jun 2018 14:34:19 -0700 Subject: [PATCH 08/29] Address comments (move heartbeater from DAGScheduler to SparkContext, move logic for getting metrics to Heartbeater), and modifiy tests for the new ExecutorMetrics format. --- .../scala/org/apache/spark/Heartbeater.scala | 17 ++- .../scala/org/apache/spark/SparkContext.scala | 25 ++++ .../org/apache/spark/executor/Executor.scala | 20 +-- .../spark/executor/ExecutorMetrics.scala | 11 +- .../apache/spark/metrics/MetricGetter.scala | 26 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 20 +-- .../scheduler/EventLoggingListener.scala | 18 ++- .../spark/scheduler/PeakExecutorMetrics.scala | 15 --- .../org/apache/spark/status/LiveEntity.scala | 2 +- .../org/apache/spark/status/api/v1/api.scala | 45 ++++++- .../org/apache/spark/util/JsonProtocol.scala | 12 +- ...ith_executor_metrics_json_expectation.json | 100 +++++++-------- .../application_1506645932520_24630151 | 20 +-- .../apache/spark/HeartbeatReceiverSuite.scala | 4 +- .../scheduler/EventLoggingListenerSuite.scala | 114 +++++++++--------- .../spark/status/AppStatusListenerSuite.scala | 47 +++----- .../apache/spark/util/JsonProtocolSuite.scala | 26 ++-- 17 files changed, 278 insertions(+), 244 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 2c89cc14189d..ca644c1a2b36 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -19,17 +19,26 @@ package org.apache.spark import java.util.concurrent.TimeUnit +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.memory.MemoryManager +import org.apache.spark.metrics.MetricGetter import org.apache.spark.util.{ThreadUtils, Utils} /** * Creates a heartbeat thread which will call the specified reportHeartbeat function at * intervals of intervalMs. * + * @param memoryManager the memory manager for execution and storage memory. * @param reportHeartbeat the heartbeat reporting function to call. * @param name the thread name for the heartbeater. * @param intervalMs the interval between heartbeats. */ -private[spark] class Heartbeater(reportHeartbeat: () => Unit, name: String, intervalMs: Long) { +private[spark] class Heartbeater( + memoryManager: MemoryManager, + reportHeartbeat: () => Unit, + name: String, + intervalMs: Long) extends Logging { // Executor for the heartbeat task private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor(name) @@ -49,5 +58,11 @@ private[spark] class Heartbeater(reportHeartbeat: () => Unit, name: String, inte heartbeater.shutdown() heartbeater.awaitTermination(10, TimeUnit.SECONDS) } + + /** Get the current metrics. */ + def getCurrentMetrics(): ExecutorMetrics = { + new ExecutorMetrics(System.currentTimeMillis(), + MetricGetter.values.map(_.getMetricValue(memoryManager)).toArray) + } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5e8595603cc9..5156e11a01ef 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -213,6 +213,7 @@ class SparkContext(config: SparkConf) extends Logging { private var _files: Seq[String] = _ private var _shutdownHookRef: AnyRef = _ private var _statusStore: AppStatusStore = _ + private var _heartbeater: Heartbeater = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -304,6 +305,11 @@ class SparkContext(config: SparkConf) extends Logging { _dagScheduler = ds } + private[spark] def heartbeater: Heartbeater = _heartbeater + private[spark] def heartbeater_=(hb: Heartbeater): Unit = { + _heartbeater = hb + } + /** * A unique identifier for the Spark application. * Its format depends on the scheduler implementation. @@ -496,6 +502,11 @@ class SparkContext(config: SparkConf) extends Logging { _dagScheduler = new DAGScheduler(this) _heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet) + // create and start the heartbeater for collecting memory metrics + _heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, "driver-heartbeater", + conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + _heartbeater.start() + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's // constructor _taskScheduler.start() @@ -1922,6 +1933,12 @@ class SparkContext(config: SparkConf) extends Logging { Utils.tryLogNonFatalError { _eventLogger.foreach(_.stop()) } + if(_heartbeater != null) { + Utils.tryLogNonFatalError { + _heartbeater.stop() + } + _heartbeater = null + } if (_dagScheduler != null) { Utils.tryLogNonFatalError { _dagScheduler.stop() @@ -2398,6 +2415,14 @@ class SparkContext(config: SparkConf) extends Logging { } } + /** Reports heartbeat metrics for the driver. */ + private def reportHeartBeat(): Unit = { + val driverUpdates = _heartbeater.getCurrentMetrics() + val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) + listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, + Some(driverUpdates))) + } + // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having finished construction. // NOTE: this must be placed at the end of the SparkContext constructor. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 43ab51482a91..0a4749e023b0 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -36,8 +36,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.memory.{MemoryManager, SparkOutOfMemoryError, TaskMemoryManager} -import org.apache.spark.metrics.MetricGetter +import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException @@ -149,8 +148,8 @@ private[spark] class Executor( private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] // Executor for the heartbeat task. - private val heartbeater = new Heartbeater(reportHeartBeat, "executor-heartbeater", - conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + private val heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, + "executor-heartbeater", conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) // must be initialized before running startDriverHeartbeat() private val heartbeatReceiverRef = @@ -774,7 +773,7 @@ private[spark] class Executor( val curGCTime = computeTotalGcTime() // get executor level memory metrics - val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager) + val executorUpdates = heartbeater.getCurrentMetrics() for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { @@ -812,15 +811,4 @@ private[spark] object Executor { // task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be // used instead. val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties] - - /** - * Get the current executor level memory metrics. - */ - def getCurrentExecutorMetrics(memoryManager: MemoryManager): ExecutorMetrics = { - val metrics = new ExecutorMetrics(System.currentTimeMillis()) - MetricGetter.idxAndValues.foreach { case (idx, metric) => - metrics.metrics(idx) = metric.getMetricValue(memoryManager) - } - metrics - } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 663aecfbebd3..9305249ccb34 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -29,8 +29,15 @@ import org.apache.spark.metrics.MetricGetter * * @param timestamp the time the metrics were collected, or -1 for Spark history * log events which are logged when a stage has completed + * @param metrics the array of executor metrics values, order and elements as + * specified in MetricGetter */ @DeveloperApi -class ExecutorMetrics private[spark] (val timestamp: Long) extends Serializable { - val metrics = new Array[Long](MetricGetter.values.length) +class ExecutorMetrics private[spark] ( + val timestamp: Long, + val metrics: Array[Long]) extends Serializable { + if (metrics.length != MetricGetter.values.length) { + throw new IllegalArgumentException("invalid metrics length " + metrics.length + + " does not equal expected length " + MetricGetter.values.length) + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala index 53c87a1261e1..27ef6d94744a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala @@ -23,7 +23,7 @@ import org.apache.spark.memory.MemoryManager sealed trait MetricGetter { def getMetricValue(memoryManager: MemoryManager): Long - val name = getClass().getName().stripSuffix("$") + val name = getClass().getName().stripSuffix("$").split("""\.""").last } abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter { @@ -53,13 +53,19 @@ case object JVMOffHeapMemory extends MetricGetter { } } -case object OnHeapExecution extends MemoryManagerMetricGetter(_.onHeapExecutionMemoryUsed) +case object OnHeapExecutionMemory extends MemoryManagerMetricGetter(_.onHeapExecutionMemoryUsed) -case object OffHeapExecution extends MemoryManagerMetricGetter(_.offHeapExecutionMemoryUsed) +case object OffHeapExecutionMemory extends MemoryManagerMetricGetter(_.offHeapExecutionMemoryUsed) -case object OnHeapStorage extends MemoryManagerMetricGetter(_.onHeapStorageMemoryUsed) +case object OnHeapStorageMemory extends MemoryManagerMetricGetter(_.onHeapStorageMemoryUsed) -case object OffHeapStorage extends MemoryManagerMetricGetter(_.offHeapStorageMemoryUsed) +case object OffHeapStorageMemory extends MemoryManagerMetricGetter(_.offHeapStorageMemoryUsed) + +case object OnHeapUnifiedMemory extends MemoryManagerMetricGetter( + (m => m.onHeapExecutionMemoryUsed + m.onHeapStorageMemoryUsed)) + +case object OffHeapUnifiedMemory extends MemoryManagerMetricGetter( + (m => m.offHeapExecutionMemoryUsed + m.offHeapStorageMemoryUsed)) case object DirectPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=direct") case object MappedPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=mapped") @@ -68,10 +74,12 @@ object MetricGetter { val values = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, - OnHeapExecution, - OffHeapExecution, - OnHeapStorage, - OffHeapStorage, + OnHeapExecutionMemory, + OffHeapExecutionMemory, + OnHeapStorageMemory, + OffHeapStorageMemory, + OnHeapUnifiedMemory, + OffHeapUnifiedMemory, DirectPoolMemory, MappedPoolMemory ) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 98ed6c75d7eb..b1a0a063680d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io.NotSerializableException -import java.lang.management.ManagementFactory import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger @@ -35,7 +34,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.{Executor, ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils @@ -210,10 +209,6 @@ class DAGScheduler( private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) - /** driver heartbeat for collecting metrics */ - private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, "driver-heartbeater", - sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) - /** * Called by the TaskSetManager to report task's starting. */ @@ -1760,20 +1755,9 @@ class DAGScheduler( messageScheduler.shutdownNow() eventProcessLoop.stop() taskScheduler.stop() - heartbeater.stop() - } - - /** Reports heartbeat metrics for the driver. */ - private def reportHeartBeat(): Unit = { - // get driver memory metrics - val driverUpdates = Executor.getCurrentExecutorMetrics(sc.env.memoryManager) - val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) - listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, - Some(driverUpdates))) - } + } eventProcessLoop.start() - heartbeater.start() } private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index baf7127a8392..f0da348a7ec0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -24,7 +24,7 @@ import java.util.EnumSet import java.util.Locale import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} @@ -84,7 +84,7 @@ private[spark] class EventLoggingListener( private val compressionCodecName = compressionCodec.map { c => CompressionCodec.getShortName(c.getClass.getName) } -logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecutorMetricsUpdates) + // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None @@ -97,8 +97,7 @@ logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecut private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) // map of live stages, to peak executor metrics for the stage - private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int), - mutable.HashMap[String, PeakExecutorMetrics]]() + private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, PeakExecutorMetrics]]() /** * Creates the log file in the configured log directory. @@ -167,7 +166,7 @@ logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecut if (shouldLogExecutorMetricsUpdates) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), - new mutable.HashMap[String, PeakExecutorMetrics]()) + new HashMap[String, PeakExecutorMetrics]()) } } @@ -190,16 +189,15 @@ logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecut liveStageExecutorMetrics.remove((event.stageInfo.stageId, attemptId)) } - // log the peak executor metrics for the stage, for each executor + // log the peak executor metrics for the stage, for each live executor, + // whether or not the executor is running tasks for the stage val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() val executorMap = liveStageExecutorMetrics.remove( (event.stageInfo.stageId, event.stageInfo.attemptNumber())) executorMap.foreach { executorEntry => { for ((executorId, peakExecutorMetrics) <- executorEntry) { - val executorMetrics = new ExecutorMetrics(-1) - System.arraycopy(peakExecutorMetrics.metrics, 0, executorMetrics.metrics, 0, - peakExecutorMetrics.metrics.size) + val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.metrics) val executorUpdate = new SparkListenerExecutorMetricsUpdate( executorId, accumUpdates, Some(executorMetrics)) logEvent(executorUpdate) @@ -346,7 +344,7 @@ private[spark] object EventLoggingListener extends Logging { private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) // A cache for compression codecs to avoid creating the same codec many times - private val codecMap = new mutable.HashMap[String, CompressionCodec] + private val codecMap = new HashMap[String, CompressionCodec] /** * Write metadata about an event log to the given stream. diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index 83bb39cf3c3e..b7c279e63b45 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.metrics.MetricGetter -import org.apache.spark.status.api.v1.PeakMemoryMetrics /** * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no @@ -49,20 +48,6 @@ private[spark] class PeakExecutorMetrics { updated } - /** - * @return None if no peak metrics have been recorded, else PeakMemoryMetrics with the peak - * values set. - */ - def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = { - if (metrics(0) < 0) { - None - } else { - val copy = new PeakMemoryMetrics - System.arraycopy(this.metrics, 0, copy.metrics, 0, this.metrics.length) - Some(copy) - } - } - /** Clears/resets the saved peak values. */ def reset(): Unit = { (0 until metrics.length).foreach { idx => metrics(idx) = 0} 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 c296f55210f2..c747ce66b78c 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -306,7 +306,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE executorLogs, memoryMetrics, blacklistedInStages, - peakExecutorMetrics.getPeakMemoryMetrics) + if (peakExecutorMetrics.metrics(0) == -1) None else Some(peakExecutorMetrics.metrics)) 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 3d6360383683..d36588ae40da 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 @@ -22,7 +22,9 @@ import java.util.Date import scala.xml.{NodeSeq, Text} import com.fasterxml.jackson.annotation.JsonIgnoreProperties -import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import com.fasterxml.jackson.core.{JsonGenerator, JsonParser} +import com.fasterxml.jackson.databind.{DeserializationContext, JsonDeserializer, JsonSerializer, SerializerProvider} +import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize} import org.apache.spark.JobExecutionStatus import org.apache.spark.metrics.MetricGetter @@ -100,7 +102,9 @@ class ExecutorSummary private[spark]( val executorLogs: Map[String, String], val memoryMetrics: Option[MemoryMetrics], val blacklistedInStages: Set[Int], - val peakMemoryMetrics: Option[PeakMemoryMetrics]) + @JsonSerialize(using = classOf[PeakMemoryMetricsSerializer]) + @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer]) + val peakMemoryMetrics: Option[Array[Long]]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, @@ -108,9 +112,40 @@ class MemoryMetrics private[spark]( val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) -class PeakMemoryMetrics private[spark]() { - // TODO special json-ification - val metrics = new Array[Long](MetricGetter.values.length) +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */ +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] { + override def deserialize( + jsonParser: JsonParser, + deserializationContext: DeserializationContext): Option[Array[Long]] = { + val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]]) + metricsMap match { + case Some(metrics) => + Some(MetricGetter.values.map { m => + metrics.getOrElse (m.name, 0L) match { + case intVal: Int => intVal.toLong + case longVal: Long => longVal + } + }.toArray) + case None => None + } + } +} + +/** serializer for peakMemoryMetrics: convert array to map with metric name as key */ +class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] { + override def serialize( + metrics: Option[Array[Long]], + jsonGenerator: JsonGenerator, + serializerProvider: SerializerProvider): Unit = { + metrics match { + case Some(m) => + val metricsMap = (0 until MetricGetter.values.length).map { idx => + MetricGetter.values (idx).name -> m(idx) + }.toMap + jsonGenerator.writeObject (metricsMap) + case None => + } + } } class JobData private[spark]( 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 37b8099f4eb4..9d4e2e3de018 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -610,12 +610,12 @@ private[spark] object JsonProtocol { */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val timeStamp = (json \ "Timestamp").extract[Long] - val metrics = new ExecutorMetrics(timeStamp) - MetricGetter.idxAndValues.foreach { case (idx, metric) => - val metricValue = (json \ metric.name).extract[Long] - metrics.metrics(idx) = metricValue - } - metrics + + val metrics = + MetricGetter.values.map {metric => + val metricVal = (json \ metric.name).extract[Long] + metricVal} + new ExecutorMetrics(timeStamp, metrics.toArray) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { 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 ac1bb97b157f..9bf2086cc8e7 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 @@ -28,16 +28,16 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 629553808, - "jvmUsedNonHeapMemory" : 205304696, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 905801, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 905801, - "offHeapUnifiedMemory" : 0, - "directMemory" : 397602, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 905801, + "JVMOffHeapMemory" : 205304696, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 905801, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 397602, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 629553808, + "OffHeapStorageMemory" : 0 } }, { "id" : "7", @@ -168,16 +168,16 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 518613056, - "jvmUsedNonHeapMemory" : 95657456, - "onHeapExecutionMemory" : 37748736, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 63104457, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 100853193, - "offHeapUnifiedMemory" : 0, - "directMemory" : 126261, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 63104457, + "JVMOffHeapMemory" : 95657456, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 100853193, + "OnHeapExecutionMemory" : 37748736, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 126261, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 518613056, + "OffHeapStorageMemory" : 0 } }, { "id" : "3", @@ -212,16 +212,16 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 726805712, - "jvmUsedNonHeapMemory" : 90709624, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 69535048, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 69535048, - "offHeapUnifiedMemory" : 0, - "directMemory" : 87796, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 69535048, + "JVMOffHeapMemory" : 90709624, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 69535048, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 87796, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 726805712, + "OffHeapStorageMemory" : 0 } }, { "id" : "2", @@ -256,16 +256,16 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 595946552, - "jvmUsedNonHeapMemory" : 91208368, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 58468944, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 58468944, - "offHeapUnifiedMemory" : 0, - "directMemory" : 87796, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 58468944, + "JVMOffHeapMemory" : 91208368, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 58468944, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 87796, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 595946552, + "OffHeapStorageMemory" : 0 } }, { "id" : "1", @@ -300,15 +300,15 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 755008624, - "jvmUsedNonHeapMemory" : 100519936, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 47962185, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 47962185, - "offHeapUnifiedMemory" : 0, - "directMemory" : 98230, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 47962185, + "JVMOffHeapMemory" : 100519936, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 47962185, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 98230, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 755008624, + "OffHeapStorageMemory" : 0 } } ] diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 index b3424f9582c0..8d132d57ef6e 100644 --- a/core/src/test/resources/spark-events/application_1506645932520_24630151 +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -33,11 +33,11 @@ {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144246,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"1920974","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"3562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"38","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1813","Value":"1812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"195602","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"3563","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1558","Value":"1557","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":3563,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":36845111,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":27318908,"Value":27318908,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":3562,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":349287,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":33,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1498974375,"Value":1498974375,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1922,"Value":1922,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":49547405,"Value":49547405,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":56,"Executor Deserialize CPU Time":49547405,"Executor Run Time":1922,"Executor CPU Time":1498974375,"Result Size":2394,"JVM GC Time":33,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":27318908,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144444,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"23365","Value":"63634","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":105575962,"Value":301246724,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":326,"Value":1107,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11931694025,"Value":31991331624,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13454,"Value":36578,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":531799977,"Value":1633060096,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":594,"Value":1930,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":594,"Executor Deserialize CPU Time":531799977,"Executor Run Time":13454,"Executor CPU Time":11931694025,"Result Size":1856,"JVM GC Time":326,"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":22950296,"Shuffle Write Time":105575962,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144840,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"24254","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":106963069,"Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":342,"Value":1449,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":12267596062,"Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13858,"Value":50436,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":519573839,"Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":573,"Value":2503,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":573,"Executor Deserialize CPU Time":519573839,"Executor Run Time":13858,"Executor CPU Time":12267596062,"Result Size":1856,"JVM GC Time":342,"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":31362123,"Shuffle Write Time":106963069,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":592412824,"JVM Used Nonheap Memory":202907152,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":905801,"Offheap Storage Memory":0,"Onheap Unified Memory":905801,"Offheap Unified Memory":0,"Direct Memory":355389,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":523121272,"JVM Used Nonheap Memory":88280720,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":52050147,"Offheap Storage Memory":0,"Onheap Unified Memory":52050147,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":214174608,"JVM Used Nonheap Memory":91548704,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47399168,"Offheap Storage Memory":0,"Onheap Unified Memory":47399168,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":518613056,"JVM Used Nonheap Memory":95657456,"Onheap Execution Memory":37748736,"Offheap Execution Memory":0,"Onheap Storage Memory":63104457,"Offheap Storage Memory":0,"Onheap Unified Memory":100853193,"Offheap Unified Memory":0,"Direct Memory":126261,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":726805712,"JVM Used Nonheap Memory":90709624,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69535048,"Offheap Storage Memory":0,"Onheap Unified Memory":69535048,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":592412824,"JVMOffHeapMemory":202907152,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":355389,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":523121272,"JVMOffHeapMemory":88280720,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":52050147,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":52050147,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":214174608,"JVMOffHeapMemory":91548704,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47399168,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47399168,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Completion Time":1524182144852,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":2503,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":50436,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":1449,"Internal":true,"Count Failed Values":true}]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145327,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"49","Value":"86","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"2002","Value":"3814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1755","Value":"3312","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":32035583,"Value":59354491,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4788,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1785119941,"Value":3284094316,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":2182,"Value":4104,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":71500541,"Value":121047946,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":136,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":136,"Executor Deserialize CPU Time":71500541,"Executor Run Time":2182,"Executor CPU Time":1785119941,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349006,"Shuffle Write Time":32035583,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145971,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"5212268","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"9572","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"121634812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"9","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1703","Value":"5517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"525948","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"9573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1609","Value":"4921","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":9573,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":97944567,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":20055909,"Value":79410400,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":9572,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":941007,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":121634816,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":7182,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":896878991,"Value":4180973307,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1722,"Value":5826,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2787355,"Value":123835301,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":195,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2787355,"Executor Run Time":1722,"Executor CPU Time":896878991,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":242714,"Shuffle Write Time":20055909,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} @@ -46,11 +46,11 @@ {"Event":"SparkListenerExecutorAdded","Timestamp":1524182149826,"Executor ID":"7","Executor Info":{"Host":"node6340.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096","stderr":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"node6340.grid.company.com","Port":5933},"Maximum Memory":956615884,"Timestamp":1524182149983,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182152418,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"48","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"6093","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"5951","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":49790497,"Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":920,"Value":1015,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9619,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5299274511,"Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":7847,"Value":13673,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687811857,"Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1037,"Value":1232,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1037,"Executor Deserialize CPU Time":687811857,"Executor Run Time":7847,"Executor CPU Time":5299274511,"Result Size":2437,"JVM GC Time":920,"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":355051,"Shuffle Write Time":49790497,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":629553808,"JVM Used Nonheap Memory":205304696,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":905801,"Offheap Storage Memory":0,"Onheap Unified Memory":905801,"Offheap Unified Memory":0,"Direct Memory":397602,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":595946552,"JVM Used Nonheap Memory":91208368,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468944,"Offheap Storage Memory":0,"Onheap Unified Memory":58468944,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":755008624,"JVM Used Nonheap Memory":100519936,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47962185,"Offheap Storage Memory":0,"Onheap Unified Memory":47962185,"Offheap Unified Memory":0,"Direct Memory":98230,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":518613056,"JVM Used Nonheap Memory":95657456,"Onheap Execution Memory":37748736,"Offheap Execution Memory":0,"Onheap Storage Memory":63104457,"Offheap Storage Memory":0,"Onheap Unified Memory":100853193,"Offheap Unified Memory":0,"Direct Memory":126261,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":726805712,"JVM Used Nonheap Memory":90709624,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69535048,"Offheap Storage Memory":0,"Onheap Unified Memory":69535048,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":629553808,"JVMOffHeapMemory":205304696,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":397602,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":595946552,"JVMOffHeapMemory":91208368,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":58468944,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":58468944,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":755008624,"JVMOffHeapMemory":100519936,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47962185,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47962185,"OffHeapUnifiedMemory":0,"DirectPoolMemory":98230,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Completion Time":1524182152419,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":1232,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":13673,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":1015,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9619,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} {"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1524182152447,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index b2d4505eaf87..9fe37510247a 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -213,8 +213,8 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) - val executorUpdates = new ExecutorMetrics(1234567L, 123456L, 543L, 12345L, 1234L, 123L, 12L, - 432L, 321L, 654L, 765L) + val executorUpdates = new ExecutorMetrics(1234567L, Array(123456L, 543L, 12345L, 1234L, 123L, + 12L, 432L, 321L, 654L, 765L)) val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId, executorUpdates)) if (executorShouldReregister) { 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 f6ad8a6f16d8..e2637ab0ab20 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -279,16 +279,20 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit Map( ((0, "1"), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))), + new ExecutorMetrics(-1L, + Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), ((0, "2"), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))), + new ExecutorMetrics(-1L, + Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), ((1, "1"), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))), + new ExecutorMetrics(-1L, + Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), ((1, "2"), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + new ExecutorMetrics(-1L, + Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) // Events to post. val events = Array( @@ -298,36 +302,50 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit createExecutorAddedEvent(2), createStageSubmittedEvent(0), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)), + new ExecutorMetrics(10L, + Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)), + new ExecutorMetrics(10L, + Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)), + new ExecutorMetrics(15L, + Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)), + new ExecutorMetrics(15L, + Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)), + new ExecutorMetrics(20L, + Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)), + new ExecutorMetrics(20L, + Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), createStageSubmittedEvent(1), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)), + new ExecutorMetrics(25L, + Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)), + new ExecutorMetrics(25L, + Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), createStageCompletedEvent(0), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)), + new ExecutorMetrics(30L, + Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)), + new ExecutorMetrics(30L, + Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)), + new ExecutorMetrics(35L, + Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)), + new ExecutorMetrics(35L, + Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)), + new ExecutorMetrics(40L, + Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), createExecutorRemovedEvent(1), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)), + new ExecutorMetrics(40L, + Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -350,21 +368,21 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(lines(0).contains("SparkListenerLogStart")) assert(lines(1).contains("SparkListenerApplicationStart")) assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) - var i = 1 + var logIdx = 1 events.foreach {event => event match { case metricsUpdate: SparkListenerExecutorMetricsUpdate => case stageCompleted: SparkListenerStageCompleted => - for (j <- 1 to 2) { - checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId, + (1 to 2).foreach { _ => + checkExecutorMetricsUpdate(lines(logIdx), stageCompleted.stageInfo.stageId, expectedMetricsEvents) - i += 1 - } - checkEvent(lines(i), event) - i += 1 + logIdx += 1 + } + checkEvent(lines(logIdx), event) + logIdx += 1 case _ => - checkEvent(lines(i), event) - i += 1 + checkEvent(lines(logIdx), event) + logIdx += 1 } } } finally { @@ -372,29 +390,24 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } - /** Create a stage submitted event for the specified stage Id. */ private def createStageSubmittedEvent(stageId: Int) = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "details")) } - /** Create a stage completed event for the specified stage Id. */ private def createStageCompletedEvent(stageId: Int) = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "details")) } - /** 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)) } - /** Create an executor added event for the specified executor Id. */ private def createExecutorRemovedEvent(executorId: Int) = { SparkListenerExecutorRemoved(0L, executorId.toString, "test") } - /** Create an executor metrics update event, with the specified executor metrics values. */ private def createExecutorMetricsUpdateEvent( executorId: Int, executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { @@ -424,34 +437,17 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit /** Check that the Spark history log line matches the expected event. */ private def checkEvent(line: String, event: SparkListenerEvent): Unit = { assert(line.contains(event.getClass.toString.split("\\.").last)) - event match { - case executorMetrics: SparkListenerExecutorMetricsUpdate => - JsonProtocol.sparkEventFromJson(parse(line)) match { - case executorMetrics2: SparkListenerExecutorMetricsUpdate => - assert(executorMetrics.execId === executorMetrics2.execId) - assert(executorMetrics2.accumUpdates.isEmpty) - checkExecutorMetrics(executorMetrics.executorUpdates, executorMetrics2.executorUpdates) - case _ => - assertTypeError("expecting SparkListenerExecutorMetricsUpdate") - } - case stageSubmitted: SparkListenerStageSubmitted => + val parsed = JsonProtocol.sparkEventFromJson(parse(line)) + assert(parsed.getClass === event.getClass) + (event, parsed) match { + case (expected: SparkListenerStageSubmitted, actual: SparkListenerStageSubmitted) => // accumulables can be different, so only check the stage Id - JsonProtocol.sparkEventFromJson(parse(line)) match { - case logStageSubmitted : SparkListenerStageSubmitted => - assert(logStageSubmitted.stageInfo.stageId == stageSubmitted.stageInfo.stageId) - case _ => - assertTypeError("expecting SparkListenerStageSubmitted") - } - case stageCompleted: SparkListenerStageCompleted => + assert(expected.stageInfo.stageId == actual.stageInfo.stageId) + case (expected: SparkListenerStageCompleted, actual: SparkListenerStageCompleted) => // accumulables can be different, so only check the stage Id - JsonProtocol.sparkEventFromJson(parse(line)) match { - case logStageSubmitted : SparkListenerStageSubmitted => - assert(logStageSubmitted.stageInfo.stageId == stageCompleted.stageInfo.stageId) - case _ => - assertTypeError("expecting SparkListenerStageCompleted") - } - case _ => - assert(JsonProtocol.sparkEventFromJson(parse(line)) === event) + assert(expected.stageInfo.stageId == actual.stageInfo.stageId) + case (expected: SparkListenerEvent, actual: SparkListenerEvent) => + assert(expected === actual) } } @@ -477,7 +473,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(false) } case _ => - assertTypeError("expecting SparkListenerExecutorMetricsUpdate") + fail("expecting SparkListenerExecutorMetricsUpdate") } } 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 136f20f8db04..a32e77305f5b 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -29,10 +29,10 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.metrics.MetricGetter import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.api.v1 -import org.apache.spark.status.api.v1.PeakMemoryMetrics import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -1218,42 +1218,42 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onExecutorAdded(createExecutorAddedEvent(2)) listener.onStageSubmitted(createStageSubmittedEvent(0)) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + new ExecutorMetrics(10L, Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + new ExecutorMetrics(10L, Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + new ExecutorMetrics(15L, Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + new ExecutorMetrics(15L, Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + new ExecutorMetrics(20L, Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + new ExecutorMetrics(20L, Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)))) listener.onStageSubmitted(createStageSubmittedEvent(1)) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + new ExecutorMetrics(25L, Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(25L, 7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + new ExecutorMetrics(25L, Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)))) listener.onStageCompleted(createStageCompletedEvent(0)) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + new ExecutorMetrics(30L, Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + new ExecutorMetrics(30L, Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + new ExecutorMetrics(35L, Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + new ExecutorMetrics(35L, Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + new ExecutorMetrics(40L, Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)))) listener.onExecutorRemoved(createExecutorRemovedEvent(1)) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + new ExecutorMetrics(40L, Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new PeakMemoryMetrics(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), - "2" -> new PeakMemoryMetrics(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) + "1" -> Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), + "2" -> Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) // check that the stored peak values match the expected values for ((id, metrics) <- expectedValues) { @@ -1261,16 +1261,9 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - assert(actual.jvmUsedHeapMemory == metrics.jvmUsedHeapMemory) - assert(actual.jvmUsedNonHeapMemory == metrics.jvmUsedNonHeapMemory) - assert(actual.onHeapExecutionMemory == metrics.onHeapExecutionMemory) - assert(actual.offHeapExecutionMemory == metrics.offHeapExecutionMemory) - assert(actual.onHeapStorageMemory == metrics.onHeapStorageMemory) - assert(actual.offHeapStorageMemory == metrics.offHeapStorageMemory) - assert(actual.onHeapUnifiedMemory == metrics.onHeapUnifiedMemory) - assert(actual.offHeapUnifiedMemory == metrics.offHeapUnifiedMemory) - assert(actual.directMemory == metrics.directMemory) - assert(actual.mappedMemory == metrics.mappedMemory) + (0 until MetricGetter.values.length).foreach { idx => + assert(actual(idx) === metrics(idx)) + } case _ => assert(false) } 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 4f3b412dbd43..8b720117a6aa 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -95,8 +95,8 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - val executorUpdates = Some(new ExecutorMetrics(1234567L, 543L, 123456L, 12345L, 1234L, 123L, - 12L, 432L, 321L, 654L, 765L)) + val executorUpdates = Some(new ExecutorMetrics(1234567L, Array(543L, 123456L, 12345L, + 1234L, 123L, 12L, 432L, 321L, 654L, 765L))) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) } val blockUpdated = @@ -876,7 +876,7 @@ private[spark] object JsonProtocolSuite extends Assertions { } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(1234567L, 123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L)) + Some(new ExecutorMetrics(1234567L, Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L))) } else { None } @@ -2073,16 +2073,16 @@ private[spark] object JsonProtocolSuite extends Assertions { | ], | "Executor Metrics Updated" : { | "Timestamp" : 1234567, - | "JVM Used Heap Memory" : 543, - | "JVM Used Nonheap Memory" : 123456, - | "Onheap Execution Memory" : 12345, - | "Offheap Execution Memory" : 1234, - | "Onheap Storage Memory" : 123, - | "Offheap Storage Memory" : 12, - | "Onheap Unified Memory" : 432, - | "Offheap Unified Memory" : 321, - | "Direct Memory" : 654, - | "Mapped Memory" : 765 + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765 | } | |} From 287133597f819417f96ae5965895c1b640703d86 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Fri, 9 Mar 2018 15:39:36 -0800 Subject: [PATCH 09/29] SPARK-23429: Add executor memory metrics to heartbeat and expose in executors REST API Add new executor level memory metrics (JVM used memory, on/off heap execution memory, on/off heap storage memory), and expose via the executors REST API. This information will help provide insight into how executor and driver JVM memory is used, and for the different memory regions. It can be used to help determine good values for spark.executor.memory, spark.driver.memory, spark.memory.fraction, and spark.memory.storageFraction. Add an ExecutorMetrics class, with jvmUsedMemory, onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, and offHeapStorageMemory. The new ExecutorMetrics will be sent by executors to the driver as part of Heartbeat. A heartbeat will be added for the driver as well, to collect these metrics for the driver. Modify the EventLoggingListener to log ExecutorMetricsUpdate events if there is a new peak value for any of the memory metrics for an executor and stage. Only the ExecutorMetrics will be logged, and not the TaskMetrics, to minimize additional logging. Modify the AppStatusListener to record the peak values for each memory metric. Add the new memory metrics to the executors REST API. --- .../org/apache/spark/HeartbeatReceiver.scala | 8 +- .../scala/org/apache/spark/Heartbeater.scala | 52 +++ .../org/apache/spark/executor/Executor.scala | 32 +- .../spark/executor/ExecutorMetrics.scala | 43 ++ .../apache/spark/memory/MemoryManager.scala | 20 + .../apache/spark/scheduler/DAGScheduler.scala | 29 +- .../scheduler/EventLoggingListener.scala | 30 +- .../spark/scheduler/PeakExecutorMetrics.scala | 106 +++++ .../spark/scheduler/SparkListener.scala | 6 +- .../spark/scheduler/TaskScheduler.scala | 10 +- .../spark/scheduler/TaskSchedulerImpl.scala | 13 +- .../spark/status/AppStatusListener.scala | 10 +- .../org/apache/spark/status/LiveEntity.scala | 9 +- .../org/apache/spark/status/api/v1/api.scala | 13 +- .../org/apache/spark/util/JsonProtocol.scala | 43 +- .../application_list_json_expectation.json | 15 + .../completed_app_list_json_expectation.json | 15 + ...ith_executor_metrics_json_expectation.json | 367 ++++++++++++++++++ .../limit_app_list_json_expectation.json | 30 +- .../minDate_app_list_json_expectation.json | 15 + .../minEndDate_app_list_json_expectation.json | 17 +- .../application_1506645932520_24630151 | 106 +++++ .../apache/spark/HeartbeatReceiverSuite.scala | 10 +- .../deploy/history/HistoryServerSuite.scala | 2 + .../spark/scheduler/DAGSchedulerSuite.scala | 7 +- .../scheduler/EventLoggingListenerSuite.scala | 166 ++++++++ .../ExternalClusterManagerSuite.scala | 4 +- .../apache/spark/util/JsonProtocolSuite.scala | 79 +++- dev/.rat-excludes | 1 + 29 files changed, 1186 insertions(+), 72 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/Heartbeater.scala create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala create mode 100644 core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json create mode 100644 core/src/test/resources/spark-events/application_1506645932520_24630151 diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index ff960b396dbf..ac6fb1f2659c 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable import scala.concurrent.Future +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ @@ -37,7 +38,8 @@ import org.apache.spark.util._ private[spark] case class Heartbeat( executorId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates - blockManagerId: BlockManagerId) + blockManagerId: BlockManagerId, + executorUpdates: ExecutorMetrics) // executor level updates /** * An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is @@ -120,14 +122,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) context.reply(true) // Messages received from executors - case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId) => + case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId, executorMetrics) => if (scheduler != null) { if (executorLastSeen.contains(executorId)) { executorLastSeen(executorId) = clock.getTimeMillis() eventLoopThread.submit(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { val unknownExecutor = !scheduler.executorHeartbeatReceived( - executorId, accumUpdates, blockManagerId) + executorId, accumUpdates, blockManagerId, executorMetrics) val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) context.reply(response) } diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala new file mode 100644 index 000000000000..b36285ed42e6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -0,0 +1,52 @@ +/* + * 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 + +import java.util.concurrent.TimeUnit + +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * Creates a heartbeat thread which will call the specified reportHeartbeat function at + * intervals of intervalMs. + * + * @param reportHeartbeat the heartbeat reporting function to call. + * @param intervalMs the interval between heartbeats. + */ +private[spark] class Heartbeater(reportHeartbeat: () => Unit, intervalMs: Long) { + // Executor for the heartbeat task + private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") + + /** Schedules a task to report a heartbeat. */ + private[spark] def start(): Unit = { + // Wait a random interval so the heartbeats don't end up in sync + val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] + + val heartbeatTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(reportHeartbeat()) + } + heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) + } + + /** Stops the heartbeat thread. */ + private[spark] def stop(): Unit = { + heartbeater.shutdown() + heartbeater.awaitTermination(10, TimeUnit.SECONDS) + } +} + diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b1856ff0f324..14966cee909a 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -148,7 +148,8 @@ private[spark] class Executor( private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] // Executor for the heartbeat task. - private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") + private val heartbeater = new Heartbeater(reportHeartBeat, + conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) // must be initialized before running startDriverHeartbeat() private val heartbeatReceiverRef = @@ -167,7 +168,7 @@ private[spark] class Executor( */ private var heartbeatFailures = 0 - startDriverHeartbeater() + heartbeater.start() private[executor] def numRunningTasks: Int = runningTasks.size() @@ -216,8 +217,7 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() - heartbeater.shutdown() - heartbeater.awaitTermination(10, TimeUnit.SECONDS) + heartbeater.stop() threadPool.shutdown() if (!isLocal) { env.stop() @@ -787,6 +787,12 @@ private[spark] class Executor( val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() val curGCTime = computeTotalGcTime() + // get executor level memory metrics + val executorUpdates = new ExecutorMetrics(System.currentTimeMillis(), + ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), + env.memoryManager.onHeapExecutionMemoryUsed, env.memoryManager.offHeapExecutionMemoryUsed, + env.memoryManager.onHeapStorageMemoryUsed, env.memoryManager.offHeapStorageMemoryUsed) + for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { taskRunner.task.metrics.mergeShuffleReadMetrics() @@ -795,7 +801,8 @@ private[spark] class Executor( } } - val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId) + val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId, + executorUpdates) try { val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( message, RpcTimeout(conf, "spark.executor.heartbeatInterval", "10s")) @@ -815,21 +822,6 @@ private[spark] class Executor( } } } - - /** - * Schedules a task to report heartbeat and partial metrics for active tasks to driver. - */ - private def startDriverHeartbeater(): Unit = { - val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") - - // Wait a random interval so the heartbeats don't end up in sync - val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] - - val heartbeatTask = new Runnable() { - override def run(): Unit = Utils.logUncaughtExceptions(reportHeartBeat()) - } - heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) - } } private[spark] object Executor { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala new file mode 100644 index 000000000000..906bdc41d05b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -0,0 +1,43 @@ +/* + * 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.executor + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Executor level metrics. + * + * This is sent to the driver periodically (on executor heartbeat), to provide + * information about each executor's metrics. + * + * @param timestamp the time the metrics were collected + * @param jvmUsedMemory the amount of JVM used memory for the executor + * @param onHeapExecutionMemory the amount of on heap execution memory used + * @param offHeapExecutionMemory the amount of off heap execution memory used + * @param onHeapStorageMemory the amount of on heap storage memory used + * @param offHeapStorageMemory the amount of off heap storage memory used + */ +@DeveloperApi +class ExecutorMetrics private[spark] ( + val timestamp: Long, + val jvmUsedMemory: Long, + val onHeapExecutionMemory: Long, + val offHeapExecutionMemory: Long, + val onHeapStorageMemory: Long, + val offHeapStorageMemory: Long) extends Serializable diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 0641adc2ab69..965515b8abe1 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -180,6 +180,26 @@ private[spark] abstract class MemoryManager( onHeapStorageMemoryPool.memoryUsed + offHeapStorageMemoryPool.memoryUsed } + /** + * On heap execution memory currently in use, in bytes. + */ + final def onHeapExecutionMemoryUsed: Long = onHeapExecutionMemoryPool.memoryUsed + + /** + * Off heap execution memory currently in use, in bytes. + */ + final def offHeapExecutionMemoryUsed: Long = offHeapExecutionMemoryPool.memoryUsed + + /** + * On heap storage memory currently in use, in bytes. + */ + final def onHeapStorageMemoryUsed: Long = onHeapStorageMemoryPool.memoryUsed + + /** + * Off heap storage memory currently in use, in bytes. + */ + final def offHeapStorageMemoryUsed: Long = offHeapStorageMemoryPool.memoryUsed + /** * Returns the execution memory consumption, in bytes, for the given task. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 041eade82d3c..4f3a0731db3b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.NotSerializableException +import java.lang.management.ManagementFactory import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger @@ -34,7 +35,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils @@ -209,6 +210,10 @@ class DAGScheduler( private[spark] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) + /** driver heartbeat for collecting metrics */ + private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, + sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + /** * Called by the TaskSetManager to report task's starting. */ @@ -246,8 +251,10 @@ class DAGScheduler( execId: String, // (taskId, stageId, stageAttemptId, accumUpdates) accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], - blockManagerId: BlockManagerId): Boolean = { - listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates)) + blockManagerId: BlockManagerId, + executorUpdates: ExecutorMetrics): Boolean = { + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, + Some(executorUpdates))) blockManagerMaster.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) } @@ -1751,9 +1758,25 @@ class DAGScheduler( messageScheduler.shutdownNow() eventProcessLoop.stop() taskScheduler.stop() + heartbeater.stop() + } + + /** Reports heartbeat metrics for the driver. */ + private def reportHeartBeat(): Unit = { + // get driver memory metrics + val driverUpdates = new ExecutorMetrics(System.currentTimeMillis(), + ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), + sc.env.memoryManager.onHeapExecutionMemoryUsed, + sc.env.memoryManager.offHeapExecutionMemoryUsed, + sc.env.memoryManager.onHeapStorageMemoryUsed, + sc.env.memoryManager.offHeapStorageMemoryUsed) + val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) + listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, + Some(driverUpdates))) } eventProcessLoop.start() + heartbeater.start() } private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 69bc51c1ecf9..e159421f4fbb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -36,6 +36,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.io.CompressionCodec @@ -93,6 +94,9 @@ private[spark] class EventLoggingListener( // Visible for tests only. private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) + // Peak metric values for each executor + private var peakExecutorMetrics = new mutable.HashMap[String, PeakExecutorMetrics]() + /** * Creates the log file in the configured log directory. */ @@ -155,7 +159,11 @@ private[spark] class EventLoggingListener( } // Events that do not trigger a flush - override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = logEvent(event) + override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { + logEvent(event) + // clear the peak metrics when a new stage starts + peakExecutorMetrics.values.foreach(_.reset()) + } override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) @@ -197,10 +205,12 @@ private[spark] class EventLoggingListener( } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { logEvent(event, flushLogger = true) + peakExecutorMetrics.put(event.executorId, new PeakExecutorMetrics()) } override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { logEvent(event, flushLogger = true) + peakExecutorMetrics.remove(event.executorId) } override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { @@ -234,8 +244,22 @@ private[spark] class EventLoggingListener( } } - // No-op because logging every update would be overkill - override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { } + /** + * Log if there is a new peak value for one of the memory metrics for the given executor. + * Metrics are cleared out when a new stage is started in onStageSubmitted, so this will + * log new peak memory metric values per executor per stage. + */ + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { + var log: Boolean = false + event.executorUpdates.foreach { executorUpdates => + val peakMetrics = peakExecutorMetrics.getOrElseUpdate(event.execId, new PeakExecutorMetrics()) + if (peakMetrics.compareAndUpdate(executorUpdates)) { + val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() + logEvent(new SparkListenerExecutorMetricsUpdate(event.execId, accumUpdates, + event.executorUpdates), flushLogger = true) + } + } + } override def onOtherEvent(event: SparkListenerEvent): Unit = { if (event.logEvent) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala new file mode 100644 index 000000000000..187cc77751e6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -0,0 +1,106 @@ +/* + * 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.scheduler + +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.status.api.v1.PeakMemoryMetrics + +/** + * Records the peak values for executor level metrics. If jvmUsedMemory is -1, then no values have + * been recorded yet. + */ +private[spark] class PeakExecutorMetrics { + private var jvmUsedMemory = -1L; + private var onHeapExecutionMemory = 0L + private var offHeapExecutionMemory = 0L + private var onHeapStorageMemory = 0L + private var offHeapStorageMemory = 0L + private var onHeapUnifiedMemory = 0L + private var offHeapUnifiedMemory = 0L + + /** + * Compare the specified memory values with the saved peak executor memory + * values, and update if there is a new peak value. + * + * @param executorMetrics the executor metrics to compare + * @return if there is a new peak value for any metric + */ + def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { + var updated: Boolean = false + + if (executorMetrics.jvmUsedMemory > jvmUsedMemory) { + jvmUsedMemory = executorMetrics.jvmUsedMemory + updated = true + } + if (executorMetrics.onHeapExecutionMemory > onHeapExecutionMemory) { + onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory + updated = true + } + if (executorMetrics.offHeapExecutionMemory > offHeapExecutionMemory) { + offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory + updated = true + } + if (executorMetrics.onHeapStorageMemory > onHeapStorageMemory) { + onHeapStorageMemory = executorMetrics.onHeapStorageMemory + updated = true + } + if (executorMetrics.offHeapStorageMemory > offHeapStorageMemory) { + offHeapStorageMemory = executorMetrics.offHeapStorageMemory + updated = true + } + val newOnHeapUnifiedMemory = (executorMetrics.onHeapExecutionMemory + + executorMetrics.onHeapStorageMemory) + if (newOnHeapUnifiedMemory > onHeapUnifiedMemory) { + onHeapUnifiedMemory = newOnHeapUnifiedMemory + updated = true + } + val newOffHeapUnifiedMemory = (executorMetrics.offHeapExecutionMemory + + executorMetrics.offHeapStorageMemory) + if ( newOffHeapUnifiedMemory > offHeapUnifiedMemory) { + offHeapUnifiedMemory = newOffHeapUnifiedMemory + updated = true + } + + updated + } + + /** + * @return None if no peak metrics have been recorded, else PeakMemoryMetrics with the peak + * values set. + */ + def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = { + if (jvmUsedMemory < 0) { + None + } else { + Some(new PeakMemoryMetrics(jvmUsedMemory, onHeapExecutionMemory, + offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory, + onHeapUnifiedMemory, offHeapUnifiedMemory)) + } + } + + /** Clears/resets the saved peak values. */ + def reset(): Unit = { + jvmUsedMemory = -1L; + onHeapExecutionMemory = 0L + offHeapExecutionMemory = 0L + onHeapStorageMemory = 0L + offHeapStorageMemory = 0L + onHeapUnifiedMemory = 0L + offHeapUnifiedMemory = 0L + } +} 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 8a112f6a37b9..effc0fa53e7c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -26,7 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo import org.apache.spark.{SparkConf, TaskEndReason} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.ui.SparkUI @@ -160,11 +160,13 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends * Periodic updates from executors. * @param execId executor id * @param accumUpdates sequence of (taskId, stageId, stageAttemptId, accumUpdates) + * @param executorUpdates executor level metrics updates */ @DeveloperApi case class SparkListenerExecutorMetricsUpdate( execId: String, - accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])]) + accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])], + executorUpdates: Option[ExecutorMetrics] = None) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 90644fea23ab..037e083c4370 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -68,14 +69,15 @@ private[spark] trait TaskScheduler { def defaultParallelism(): Int /** - * Update metrics for in-progress tasks and let the master know that the BlockManager is still - * alive. Return true if the driver knows about the given block manager. Otherwise, return false, - * indicating that the block manager should re-register. + * Update metrics for in-progress tasks and executor metrics, and let the master know that the + * BlockManager is still alive. Return true if the driver knows about the given block manager. + * Otherwise, return false, indicating that the block manager should re-register. */ def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean + blockManagerId: BlockManagerId, + executorUpdates: ExecutorMetrics): Boolean /** * Get an application ID associated with the job. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 598b62f85a1f..cd9bc5f51e1d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -28,6 +28,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -435,14 +436,15 @@ private[spark] class TaskSchedulerImpl( } /** - * Update metrics for in-progress tasks and let the master know that the BlockManager is still - * alive. Return true if the driver knows about the given block manager. Otherwise, return false, - * indicating that the block manager should re-register. + * Update metrics for in-progress tasks and executor metrics, and let the master know that the + * BlockManager is still alive. Return true if the driver knows about the given block manager. + * Otherwise, return false, indicating that the block manager should re-register. */ override def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean = { + blockManagerId: BlockManagerId, + executorMetrics: ExecutorMetrics): Boolean = { // (taskId, stageId, stageAttemptId, accumUpdates) val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { accumUpdates.flatMap { case (id, updates) => @@ -452,7 +454,8 @@ private[spark] class TaskSchedulerImpl( } } } - dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId) + dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId, + executorMetrics) } def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized { 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 5ea161cd0d15..6a54a589b4f7 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1 @@ -669,6 +669,14 @@ private[spark] class AppStatusListener( } } } + event.executorUpdates.foreach { updates: ExecutorMetrics => + // check if there is a new peak value for any of the executor level memory metrics + liveExecutors.get(event.execId).foreach { exec: LiveExecutor => + if (exec.peakExecutorMetrics.compareAndUpdate(updates)) { + maybeUpdate(exec, now) + } + } + } } override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = { 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 79e3f13b826c..afb3e60e0b28 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -27,7 +27,7 @@ import com.google.common.collect.Interners import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, TaskInfo} +import org.apache.spark.scheduler.{AccumulableInfo, PeakExecutorMetrics, StageInfo, TaskInfo} import org.apache.spark.status.api.v1 import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.SparkUI @@ -268,6 +268,9 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE def hasMemoryInfo: Boolean = totalOnHeap >= 0L + // peak values for executor level metrics + var peakExecutorMetrics = new PeakExecutorMetrics + def hostname: String = if (host != null) host else hostPort.split(":")(0) override protected def doUpdate(): Any = { @@ -302,10 +305,10 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE Option(removeReason), executorLogs, memoryMetrics, - blacklistedInStages) + blacklistedInStages, + peakExecutorMetrics.getPeakMemoryMetrics) new ExecutorSummaryWrapper(info) } - } private class LiveExecutorStageSummary( 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 971d7e90fa7b..4d14ba7560e8 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 @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.JobExecutionStatus +import org.apache.spark.executor.ExecutorMetrics case class ApplicationInfo private[spark]( id: String, @@ -98,7 +99,8 @@ class ExecutorSummary private[spark]( val removeReason: Option[String], val executorLogs: Map[String, String], val memoryMetrics: Option[MemoryMetrics], - val blacklistedInStages: Set[Int]) + val blacklistedInStages: Set[Int], + val peakMemoryMetrics: Option[PeakMemoryMetrics]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, @@ -106,6 +108,15 @@ class MemoryMetrics private[spark]( val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) +class PeakMemoryMetrics private[spark]( + val jvmUsedMemory: Long, + val onHeapExecutionMemory: Long, + val offHeapExecutionMemory: Long, + val onHeapStorageMemory: Long, + val offHeapStorageMemory: Long, + val onHeapUnifiedMemory: Long, + val offHeapUnifiedMemory: Long) + class JobData private[spark]( val jobId: Int, val name: String, 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 50c6461373de..c2625efbcf55 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -236,6 +236,7 @@ private[spark] object JsonProtocol { def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = { val execId = metricsUpdate.execId val accumUpdates = metricsUpdate.accumUpdates + val executorMetrics = metricsUpdate.executorUpdates.map(executorMetricsToJson(_)) ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.metricsUpdate) ~ ("Executor ID" -> execId) ~ ("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) => @@ -243,7 +244,9 @@ private[spark] object JsonProtocol { ("Stage ID" -> stageId) ~ ("Stage Attempt ID" -> stageAttemptId) ~ ("Accumulator Updates" -> JArray(updates.map(accumulableInfoToJson).toList)) - }) + }) ~ + ("Executor Metrics Updated" -> executorMetrics) + } def blockUpdateToJson(blockUpdate: SparkListenerBlockUpdated): JValue = { @@ -379,6 +382,21 @@ private[spark] object JsonProtocol { ("Updated Blocks" -> updatedBlocks) } + /** + * Convert ExecutorMetrics to JSON. + * + * @param executorMetrics the executor metrics + * @return the JSON representation + */ + def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { + ("Timestamp" -> executorMetrics.timestamp) ~ + ("JVM Used Memory" -> executorMetrics.jvmUsedMemory) ~ + ("Onheap Execution Memory" -> executorMetrics.onHeapExecutionMemory) ~ + ("Offheap Execution Memory" -> executorMetrics.offHeapExecutionMemory) ~ + ("Onheap Storage Memory" -> executorMetrics.onHeapStorageMemory) ~ + ("Offheap Storage Memory" -> executorMetrics.offHeapStorageMemory) + } + def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { val reason = Utils.getFormattedClassName(taskEndReason) val json: JObject = taskEndReason match { @@ -585,6 +603,23 @@ private[spark] object JsonProtocol { SparkListenerTaskGettingResult(taskInfo) } + /** + * Extract the ExecutorMetrics from JSON. + * + * @param json the JSON representation of executor metrics + * @return the ExecutorMetrics + */ + def executorMetricsFromJson(json: JValue): ExecutorMetrics = { + val timeStamp = (json \ "Timestamp").extract[Long] + val jvmUsedMemory = (json \ "JVM Used Memory").extract[Long] + val onHeapExecutionMemory = (json \ "Onheap Execution Memory").extract[Long] + val offHeapExecutionMemory = (json \ "Offheap Execution Memory").extract[Long] + val onHeapStorageMemory = (json \ "Onheap Storage Memory").extract[Long] + val offHeapStorageMemory = (json \ "Offheap Storage Memory").extract[Long] + new ExecutorMetrics(timeStamp, jvmUsedMemory, onHeapExecutionMemory, + offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory) + } + def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { val stageId = (json \ "Stage ID").extract[Int] val stageAttemptId = @@ -691,7 +726,11 @@ private[spark] object JsonProtocol { (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) (taskId, stageId, stageAttemptId, updates) } - SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates) + val executorUpdates = jsonOption(json \ "Executor Metrics Updated") match { + case None => None + case Some(executorUpdate) => Some(executorMetricsFromJson(executorUpdate)) + } + SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates) } def blockUpdateFromJson(json: JValue): SparkListenerBlockUpdated = { 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 4fecf84db65a..9858c73fae2f 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index 4fecf84db65a..94eddfa0c7bb 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "attempts" : [ { 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 new file mode 100644 index 000000000000..1cd5fbb794cb --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json @@ -0,0 +1,367 @@ +[ { + "id" : "driver", + "hostPort" : "cluster-node0033.company.com:53121", + "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" : 1046269132, + "addTime" : "2018-03-19T18:21:24.751GMT", + "executorLogs" : { }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 1046269132, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 671982856, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 900126, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 900126, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "8", + "hostPort" : "cluster-node6128.company.com:1664", + "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" : 956615884, + "addTime" : "2018-03-19T18:22:25.988GMT", + "executorLogs" : { + "stdout" : "http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 226589176, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 0, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 0, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "7", + "hostPort" : "cluster-node2449.company.com:26914", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 11073, + "totalGCTime" : 942, + "totalInputBytes" : 36849246, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 349006, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:22:10.182GMT", + "executorLogs" : { + "stdout" : "http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 365803960, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 562630, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 562630, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "6", + "hostPort" : "cluster-node6148.company.com:1233", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 34956, + "totalGCTime" : 5484, + "totalInputBytes" : 24250210, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 242714, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:22:19.537GMT", + "executorLogs" : { + "stdout" : "http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 501950360, + "onHeapExecutionMemory" : 35651584, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 562630, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 36214214, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "5", + "hostPort" : "cluster-node5882.company.com:24590", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 30945, + "totalGCTime" : 6111, + "totalInputBytes" : 36838295, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 355051, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:22:11.267GMT", + "executorLogs" : { + "stdout" : "http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 513138800, + "onHeapExecutionMemory" : 41943040, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 562630, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 42505670, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "4", + "hostPort" : "cluster-node6137.company.com:19036", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 79365, + "totalGCTime" : 6979, + "totalInputBytes" : 50409514, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 31362123, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:21:44.950GMT", + "executorLogs" : { + "stdout" : "http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 707255176, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 83580552, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 83580552, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "3", + "hostPort" : "cluster-node1177.company.com:12614", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 2, + "totalTasks" : 2, + "totalDuration" : 38198, + "totalGCTime" : 1024, + "totalInputBytes" : 50423609, + "totalShuffleRead" : 438675, + "totalShuffleWrite" : 26424033, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:21:33.370GMT", + "executorLogs" : { + "stdout" : "http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 620936920, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 72624976, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 72624976, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "2", + "hostPort" : "cluster-node6036.company.com:17435", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 44061, + "totalGCTime" : 4677, + "totalInputBytes" : 50423423, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 22950296, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:21:41.213GMT", + "executorLogs" : { + "stdout" : "http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 695372944, + "onHeapExecutionMemory" : 0, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 58468552, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 58468552, + "offHeapUnifiedMemory" : 0 + } +}, { + "id" : "1", + "hostPort" : "cluster-node4342.company.com:29144", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 2, + "totalTasks" : 2, + "totalDuration" : 31501, + "totalGCTime" : 936, + "totalInputBytes" : 74654808, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 20352030, + "isBlacklisted" : false, + "maxMemory" : 956615884, + "addTime" : "2018-03-19T18:21:33.307GMT", + "executorLogs" : { + "stdout" : "http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stdout?start=-4096", + "stderr" : "http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 956615884, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "jvmUsedMemory" : 505317000, + "onHeapExecutionMemory" : 35651584, + "offHeapExecutionMemory" : 0, + "onHeapStorageMemory" : 47961406, + "offHeapStorageMemory" : 0, + "onHeapUnifiedMemory" : 83612990, + "offHeapUnifiedMemory" : 0 + } +} ] 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 79950b0dc648..bffe37ce52b4 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "attempts" : [ { @@ -28,19 +43,4 @@ "startTimeEpoch" : 1515492942372, "endTimeEpoch" : 1515493477606 } ] -}, { - "id" : "app-20161116163331-0000", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2016-11-16T22:33:29.916GMT", - "endTime" : "2016-11-16T22:33:40.587GMT", - "lastUpdated" : "", - "duration" : 10671, - "sparkUser" : "jose", - "completed" : true, - "appSparkVersion" : "2.1.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1479335609916, - "endTimeEpoch" : 1479335620587 - } ] } ] 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 7d60977dcd4f..066217dacbe9 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "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 dfbfd8aedcc2..b1226c905d6a 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1506645932520_24630151", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2018-03-19T18:21:00.843GMT", + "endTime" : "2018-03-19T18:23:12.501GMT", + "lastUpdated" : "", + "duration" : 131658, + "sparkUser" : "edlu", + "completed" : true, + "appSparkVersion" : "2.4.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1521483660843, + "endTimeEpoch" : 1521483792501 + } ] +}, { "id" : "application_1516285256255_0012", "name" : "Spark shell", "attempts" : [ { @@ -101,4 +116,4 @@ "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890 } ] -} ] \ No newline at end of file +} ] diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 new file mode 100644 index 000000000000..26289113d369 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -0,0 +1,106 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.4.0-SNAPSHOT"} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483677960,"JVM Used Memory":587908264,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"cluster-node0033.company.com","Port":53121},"Maximum Memory":1046269132,"Timestamp":1521483684751,"Maximum Onheap Memory":1046269132,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_31/jre","Java Version":"1.8.0_31 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.jars.ivySettings":"/export/apps/spark/commonconf/ivysettings.xml","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.driver.host":"cluster-node0033.company.com","spark.dynamicAllocation.sustainedSchedulerBacklogTimeout":"5","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"36653","spark.shuffle.service.enabled":"true","spark.ui.acls.enable":"true","spark.reducer.maxSizeInFlight":"48m","spark.yarn.queue":"spark_default","spark.repl.class.uri":"spark://cluster-node0033.company.com:36653/classes","spark.jars":"","spark.yarn.historyServer.address":"clustersh01.company.com:18080","spark.memoryOverhead.multiplier.percent":"10","spark.repl.class.outputDir":"/grid/a/mapred/tmp/spark-c015ad60-cb0b-4920-a14d-e79aa91000f3/repl-c23128a3-a417-40f7-933e-3cd0a5e7c747","spark.dynamicAllocation.cachedExecutorIdleTimeout":"1200","spark.yarn.access.namenodes":"hdfs://clusternn02.company.com:9000","spark.app.name":"Spark shell","spark.dynamicAllocation.schedulerBacklogTimeout":"5","spark.yarn.security.credentials.hive.enabled":"false","spark.yarn.am.cores":"1","spark.memoryOverhead.min":"384","spark.scheduler.mode":"FIFO","spark.driver.memory":"2G","spark.executor.instances":"4","spark.isolated.classloader.additional.classes.prefix":"com_company_","spark.logConf":"true","spark.ui.showConsoleProgress":"true","spark.user.priority.jars":"*********(redacted)","spark.isolated.classloader":"true","spark.sql.sources.schemaStringLengthThreshold":"40000","spark.yarn.secondary.jars":"spark-avro_2.11-3.2.0.21.jar,grid-topology-1.0.jar","spark.reducer.maxBlocksInFlightPerAddress":"100","spark.dynamicAllocation.maxExecutors":"900","spark.yarn.appMasterEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.executor.id":"driver","spark.yarn.am.memory":"2G","spark.driver.cores":"1","spark.min.mem.vore.ratio":"5","spark.sql.sources.partitionOverwriteMode":"DYNAMIC","spark.submit.deployMode":"client","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.isolated.classloader.default.jar":"*dali-data-spark*","spark.authenticate":"true","spark.eventLog.usexattr":"true","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.memory":"2G","spark.home":"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51","spark.reducer.maxReqsInFlight":"10","spark.eventLog.dir":"hdfs://clusternn02.company.com:9000/system/spark-history","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.isolated.classes":"org.apache.hadoop.hive.ql.io.CombineHiveInputFormat$CombineHiveInputSplit","spark.eventLog.compress":"true","spark.executor.cores":"1","spark.version":"2.1.0","spark.driver.appUIAddress":"http://cluster-node0033.company.com:46527","spark.repl.local.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"clusterwp01.company.com","spark.min.memory-gb.size":"10","spark.dynamicAllocation.minExecutors":"1","spark.dynamicAllocation.initialExecutors":"3","spark.expressionencoder.org.apache.avro.specific.SpecificRecord":"com.databricks.spark.avro.AvroEncoder$","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://clusterwp01.company.com:8080/proxy/application_1506645932520_24630151","spark.executorEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.dynamicAllocation.executorIdleTimeout":"150","spark.shell.auto.node.labeling":"true","spark.yarn.dist.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.app.id":"application_1506645932520_24630151","spark.ui.view.acls":"*"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_31/jre/lib/amd64","user.dir":"*********(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.31-b07","java.endorsed.dirs":"/usr/java/jdk1.8.0_31/jre/lib/endorsed","java.runtime.version":"1.8.0_31-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_31/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_31/jre/lib/resources.jar:/usr/java/jdk1.8.0_31/jre/lib/rt.jar:/usr/java/jdk1.8.0_31/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_31/jre/lib/jsse.jar:/usr/java/jdk1.8.0_31/jre/lib/jce.jar:/usr/java/jdk1.8.0_31/jre/lib/charsets.jar:/usr/java/jdk1.8.0_31/jre/lib/jfr.jar:/usr/java/jdk1.8.0_31/jre/classes","file.encoding":"ANSI_X3.4-1968","user.timezone":"*********(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"2.6.32-504.16.2.el6.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*********(redacted)","sun.jnu.encoding":"ANSI_X3.4-1968","user.language":"*********(redacted)","java.vendor.url":"*********(redacted)","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":"*********(redacted)","user.name":"*********(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --class org.apache.spark.repl.Main --name Spark shell --jars /export/home/edlu/spark-avro_2.11-3.2.0.21.jar,/export/apps/hadoop/site/lib/grid-topology-1.0.jar --num-executors 4 spark-shell","java.home":"/usr/java/jdk1.8.0_31/jre","java.version":"1.8.0_31","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-servlet-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/derby-10.12.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-reflect-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-graphx_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-util-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/base64-2.3.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-auth-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/validation-api-1.1.0.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-api-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/objenesis-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/conf/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpclient-4.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/kryo-shaded-3.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-library-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-net-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xz-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-annotations-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/activation-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arpack_combined_all-0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libthrift-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aircompressor-0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-jackson-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/asm-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-hive_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ivy-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/export/apps/hadoop/site/etc/hadoop/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-java-1.1.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-format-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-all-4.1.17.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-ipc-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xmlenc-0.52.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jdo-api-3.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-client-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-runtime-3.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/pyrolite-4.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-catalyst_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-collections-3.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-api-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stream-2.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-format-2.3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-vector-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-web-proxy-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sketch_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-common-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hppc-0.7.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sql_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/univocity-parsers-2.5.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-math3-3.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compiler-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-1.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/java-xmlbuilder-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-annotations-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-3.9.9.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zookeeper-3.4.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-compiler-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/eigenbase-properties-1.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-yarn_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/JavaEWAH-0.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsr305-1.3.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libfb303-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.annotation-api-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-digester-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-jvm-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-framework-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/paranamer-2.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/janino-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-core-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-core-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsch-0.1.42.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-unsafe_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-codec-1.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jtransforms-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/lz4-java-1.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-core-3.2.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/core-1.1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/leveldbjni-all-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-dbcp-1.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang3-3.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill-java-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jodd-core-3.5.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-pool-1.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/minlog-1.3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/gson-2.2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/py4j-0.10.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-streaming_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/machinist_2.11-0.6.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-0.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-app-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-graphite-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-core-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib-local_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-memory-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-guava-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xercesImpl-2.9.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-tags_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javolution-5.5.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/joda-time-2.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-2.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-jobclient-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang-2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/compress-lzf-1.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-crypto-1.0.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-core-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-recipes-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guava-14.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-core_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-common_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-launcher_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr4-runtime-4.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-util-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jaxb-api-2.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-io-2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-encoding-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpcore-4.4.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-xc-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/protobuf-java-2.5.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-configuration-1.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compress-1.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-mapreduce-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ST4-4.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-core-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-shuffle-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-repl_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/opencsv-2.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-logging-1.1.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-cli-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-client-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-hdfs-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/log4j-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-column-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill_2.11-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stringtemplate-3.2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-common-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-shuffle_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-kvstore_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jta-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javassist-3.18.1-GA.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-httpclient-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jets3t-0.9.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-json-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bcprov-jdk15on-1.58.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/oro-2.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsp-api-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1506645932520_24630151","Timestamp":1521483660843,"User":"edlu"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483693307,"Executor ID":"1","Executor Info":{"Host":"cluster-node4342.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stdout?start=-4096","stderr":"http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483693370,"Executor ID":"3","Executor Info":{"Host":"cluster-node1177.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stdout?start=-4096","stderr":"http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"cluster-node4342.company.com","Port":29144},"Maximum Memory":956615884,"Timestamp":1521483693405,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"cluster-node1177.company.com","Port":12614},"Maximum Memory":956615884,"Timestamp":1521483693483,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483701213,"Executor ID":"2","Executor Info":{"Host":"cluster-node6036.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stdout?start=-4096","stderr":"http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"cluster-node6036.company.com","Port":17435},"Maximum Memory":956615884,"Timestamp":1521483701726,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483704950,"Executor ID":"4","Executor Info":{"Host":"cluster-node6137.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stdout?start=-4096","stderr":"http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"cluster-node6137.company.com","Port":19036},"Maximum Memory":956615884,"Timestamp":1521483705369,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line44.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line44.$read$$iw$$iw$$iw$$iw.(:59)\n$line44.$read$$iw$$iw$$iw.(:61)\n$line44.$read$$iw$$iw.(:63)\n$line44.$read$$iw.(:65)\n$line44.$read.(:67)\n$line44.$read$.(:71)\n$line44.$read$.()\n$line44.$eval$.$print$lzycompute(:7)\n$line44.$eval$.$print(:6)\n$line44.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1521483706188} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1521483706193} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483707958,"JVM Used Memory":511901456,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":415523,"Offheap Storage Memory":0}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line48.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line48.$read$$iw$$iw$$iw$$iw.(:59)\n$line48.$read$$iw$$iw$$iw.(:61)\n$line48.$read$$iw$$iw.(:63)\n$line48.$read$$iw.(:65)\n$line48.$read.(:67)\n$line48.$read$.(:71)\n$line48.$read$.()\n$line48.$eval$.$print$lzycompute(:7)\n$line48.$eval$.$print(:6)\n$line48.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1521483708914} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1521483708914} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483712396,"JVM Used Memory":197797104,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at :40","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- AnalysisBarrier\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nappId: string, attemptId: string, name: string, mode: string, completed: string, duration: string, endTime: string, endTimeEpoch: string, lastUpdated: string, lastUpdatedEpoch: string, sparkUser: string, startTime: string, startTimeEpoch: string, appSparkVersion: string, endDate: string, azkaban.link.workflow.url: string, azkaban.link.execution.url: string, azkaban.link.job.url: string, user.name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@babef5e, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) LocalLimit 21\n +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@babef5e, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"LocalLimit","simpleString":"LocalLimit 21","children":[{"nodeName":"Project","simpleString":"Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"InMemoryTableScan","simpleString":"InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]","children":[],"metrics":[{"name":"number of output rows","accumulatorId":35,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":36,"metricType":"timing"}]}],"metrics":[]}],"metrics":[]}],"metrics":[]}],"metrics":[{"name":"duration total (min, med, max)","accumulatorId":34,"metricType":"timing"}]}],"metrics":[]},"time":1521483710146} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1521483710385,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]}],"Stage IDs":[0,1,2],"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710427,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710514,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1521483710517,"Executor ID":"2","Host":"cluster-node6036.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"4","Host":"cluster-node6137.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483715514,"JVM Used Memory":83495872,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":36413,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483720047,"JVM Used Memory":121617408,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":36413,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483717971,"JVM Used Memory":603050152,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483722392,"JVM Used Memory":256639776,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483724389,"JVM Used Memory":431714816,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483725467,"JVM Used Memory":504156608,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":25165398,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483730046,"JVM Used Memory":292871096,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483727958,"JVM Used Memory":607069712,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1521483728324,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483728367,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"154334487","Value":"154334486","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"27714","Value":"27713","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":37809697,"Value":37809697,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":98699361,"Value":98699361,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":20002743,"Value":20002743,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":834,"Value":834,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":1856,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":13106272441,"Value":13106272441,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":15665,"Value":15665,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":741777109,"Value":741777109,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":2081,"Value":2081,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2081,"Executor Deserialize CPU Time":741777109,"Executor Run Time":15665,"Executor CPU Time":13106272441,"Result Size":1856,"JVM GC Time":834,"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":20002743,"Shuffle Write Time":98699361,"Shuffle Records Written":466636},"Input Metrics":{"Bytes Read":37809697,"Records Read":466636},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483732392,"JVM Used Memory":207788120,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":21875994,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483730182,"Executor ID":"7","Executor Info":{"Host":"cluster-node2449.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stdout?start=-4096","stderr":"http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1521483730192,"Executor ID":"7","Host":"cluster-node2449.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"cluster-node2449.company.com","Port":26914},"Maximum Memory":956615884,"Timestamp":1521483730295,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483731267,"Executor ID":"5","Executor Info":{"Host":"cluster-node5882.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stdout?start=-4096","stderr":"http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1521483731270,"Executor ID":"5","Host":"cluster-node5882.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"cluster-node5882.company.com","Port":24590},"Maximum Memory":956615884,"Timestamp":1521483731726,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483734387,"JVM Used Memory":585461008,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483735466,"JVM Used Memory":286822456,"Onheap Execution Memory":35651584,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483737958,"JVM Used Memory":619161088,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483739537,"Executor ID":"6","Executor Info":{"Host":"cluster-node6148.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stdout?start=-4096","stderr":"http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1521483739540,"Executor ID":"6","Host":"cluster-node6148.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"cluster-node6148.company.com","Port":1233},"Maximum Memory":956615884,"Timestamp":1521483739859,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483742392,"JVM Used Memory":618421104,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":72624976,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1521483730192,"Executor ID":"7","Host":"cluster-node2449.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483741265,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"1953294","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"3575","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"76","Value":"75","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"7964","Value":"7963","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"196587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"3575","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"7495","Value":"7494","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":3575,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":36849246,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":71122623,"Value":71122623,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":3575,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":349006,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":942,"Value":942,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":2437,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5608539520,"Value":5608539520,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":9646,"Value":9646,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":775466204,"Value":775466204,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1177,"Value":1177,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1177,"Executor Deserialize CPU Time":775466204,"Executor Run Time":9646,"Executor CPU Time":5608539520,"Result Size":2437,"JVM GC Time":942,"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":349006,"Shuffle Write Time":71122623,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483744387,"JVM Used Memory":533322584,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":52049755,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1521483728324,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483741988,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"63","Value":"138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"13344","Value":"21307","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"12907","Value":"20401","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":43359041,"Value":114481664,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":102,"Value":1044,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4831,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":2885767088,"Value":8494306608,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":13538,"Value":23184,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":90660797,"Value":866127001,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":104,"Value":1281,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":104,"Executor Deserialize CPU Time":90660797,"Executor Run Time":13538,"Executor CPU Time":2885767088,"Result Size":2394,"JVM GC Time":102,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":43359041,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483745464,"JVM Used Memory":504693976,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483746631,"JVM Used Memory":365802552,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1521483745988,"Executor ID":"8","Executor Info":{"Host":"cluster-node6128.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stdout?start=-4096","stderr":"http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"8","Host":"cluster-node6128.company.com","Port":1664},"Maximum Memory":956615884,"Timestamp":1521483746284,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483746534,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"206421303","Value":"360755789","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"65190","Value":"92903","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423609,"Value":88233306,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":123692332,"Value":222391693,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":26424033,"Value":46426776,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":878,"Value":1712,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":3712,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":19065776829,"Value":32172049270,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":35010,"Value":50675,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":732371255,"Value":1474148364,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":942,"Value":3023,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":942,"Executor Deserialize CPU Time":732371255,"Executor Run Time":35010,"Executor CPU Time":19065776829,"Result Size":1856,"JVM GC Time":878,"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":26424033,"Shuffle Write Time":123692332,"Shuffle Records Written":624246},"Input Metrics":{"Bytes Read":50423609,"Records Read":624246},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483750045,"JVM Used Memory":207536536,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":2461716,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483747958,"JVM Used Memory":641729640,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"5","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483751931,"JVM Used Memory":513138800,"Onheap Execution Memory":65536,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483752392,"JVM Used Memory":619676000,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483754387,"JVM Used Memory":695372944,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468552,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483755464,"JVM Used Memory":504694360,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483756630,"JVM Used Memory":365802904,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1521483710517,"Executor ID":"2","Host":"cluster-node6036.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483754578,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"73812","Value":"166715","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":214358240,"Value":436749933,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":3,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":4677,"Value":6389,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":24023445128,"Value":56195494398,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":42072,"Value":92747,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":952244001,"Value":2426392365,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":1796,"Value":4819,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1796,"Executor Deserialize CPU Time":952244001,"Executor Run Time":42072,"Executor CPU Time":24023445128,"Result Size":1856,"JVM GC Time":4677,"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":22950296,"Shuffle Write Time":214358240,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"6","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483758435,"JVM Used Memory":397749936,"Onheap Execution Memory":65536,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483760046,"JVM Used Memory":488354552,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":25910376,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483757958,"JVM Used Memory":662984224,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483762392,"JVM Used Memory":619676384,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"5","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483762915,"JVM Used Memory":353487552,"Onheap Execution Memory":41943040,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1521483731270,"Executor ID":"5","Host":"cluster-node5882.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483762215,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"5784372","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"10678","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"125829116","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"131","Value":"269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"22823","Value":"44130","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"586742","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"10679","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"19841","Value":"40242","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":10679,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":110532652,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":215191844,"Value":329673508,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":10678,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1053344,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":125829120,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":9,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":6111,"Value":7155,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":7268,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":8017651595,"Value":16511958203,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":27779,"Value":50963,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1058013355,"Value":1924140356,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":2357,"Value":3638,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2357,"Executor Deserialize CPU Time":1058013355,"Executor Run Time":27779,"Executor CPU Time":8017651595,"Result Size":2437,"JVM GC Time":6111,"Result Serialization Time":9,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355051,"Shuffle Write Time":215191844,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483765464,"JVM Used Memory":504694712,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483766630,"JVM Used Memory":365803256,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483767180,"JVM Used Memory":226589112,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"6","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483768433,"JVM Used Memory":501950360,"Onheap Execution Memory":35651584,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483770047,"JVM Used Memory":180202344,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":55709919,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483767958,"JVM Used Memory":665764240,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483772392,"JVM Used Memory":619676736,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483775464,"JVM Used Memory":504695064,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483776630,"JVM Used Memory":365803608,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1521483739540,"Executor ID":"6","Host":"cluster-node6148.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483774496,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"300","Value":"569","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"26291","Value":"70421","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"24092","Value":"64334","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":521474789,"Value":851148297,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":13,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":5484,"Value":12639,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9705,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":7239844597,"Value":23751802800,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":30603,"Value":81566,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1062050912,"Value":2986191268,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3790,"Value":7428,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3790,"Executor Deserialize CPU Time":1062050912,"Executor Run Time":30603,"Executor CPU Time":7239844597,"Result Size":2437,"JVM GC Time":5484,"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":242714,"Shuffle Write Time":521474789,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710514,"Completion Time":1521483774499,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":7428,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":23751802800,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":81566,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":851148297,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":12639,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2986191268,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"70421","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9705,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"569","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"64334","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483777175,"JVM Used Memory":226589144,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483780046,"JVM Used Memory":639104112,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":83580552,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483777958,"JVM Used Memory":669979432,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483782392,"JVM Used Memory":620936568,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483785464,"JVM Used Memory":505317000,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483786630,"JVM Used Memory":365803960,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483787175,"JVM Used Memory":226589176,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483790047,"JVM Used Memory":707255176,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69534656,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483787958,"JVM Used Memory":671982856,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483792392,"JVM Used Memory":620936920,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"4","Host":"cluster-node6137.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483789895,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"131866","Value":"298581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":464179545,"Value":900929478,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":6979,"Value":13368,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":25321337014,"Value":81516831412,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":74896,"Value":167643,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1173542343,"Value":3599934708,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":4315,"Value":9134,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4315,"Executor Deserialize CPU Time":1173542343,"Executor Run Time":74896,"Executor CPU Time":25321337014,"Result Size":1856,"JVM GC Time":6979,"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":31362123,"Shuffle Write Time":464179545,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710427,"Completion Time":1521483789896,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3599934708,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"298581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":81516831412,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":9134,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":900929478,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":167643,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":13368,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483789907,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1521483789924,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483794387,"JVM Used Memory":317344968,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468552,"Offheap Storage Memory":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1521483789924,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1521483792118,"Failed":false,"Killed":false,"Accumulables":[{"ID":34,"Name":"duration total (min, med, max)","Update":"1","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":35,"Name":"number of output rows","Update":"6928","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Update":"1181","Value":"1180","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"number of output rows","Update":"10945","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"number of output rows","Update":"62","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Update":"33619967","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"sort time total (min, med, max)","Update":"6","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":22,"Name":"duration total (min, med, max)","Update":"896","Value":"895","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"peak memory total (min, med, max)","Update":"34078719","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Update":"53","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"duration total (min, med, max)","Update":"1109","Value":"1108","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Update":11007,"Value":11007,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":122827,"Value":122827,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":315848,"Value":315848,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":7,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Update":67698688,"Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":92,"Name":"internal.metrics.jvmGCTime","Update":146,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Update":4685,"Value":4685,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Update":970272606,"Value":970272606,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.executorRunTime","Update":1923,"Value":1923,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Update":162560302,"Value":162560302,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Update":230,"Value":230,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":230,"Executor Deserialize CPU Time":162560302,"Executor Run Time":1923,"Executor CPU Time":970272606,"Result Size":4685,"JVM GC Time":146,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":7,"Local Blocks Fetched":1,"Fetch Wait Time":1,"Remote Bytes Read":315848,"Remote Bytes Read To Disk":0,"Local Bytes Read":122827,"Total Records Read":11007},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":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":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483789907,"Completion Time":1521483792119,"Accumulables":[{"ID":92,"Name":"internal.metrics.jvmGCTime","Value":146,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Value":11007,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"number of output rows","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":89,"Name":"internal.metrics.executorRunTime","Value":1923,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":7,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"number of output rows","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":315848,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Value":4685,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"peak memory total (min, med, max)","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"duration total (min, med, max)","Value":"895","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Value":162560302,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"duration total (min, med, max)","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Value":"1180","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"sort time total (min, med, max)","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Value":230,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Value":970272606,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"number of output rows","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":122827,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"duration total (min, med, max)","Value":"1108","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1521483792127,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1521483792153} +{"Event":"SparkListenerUnpersistRDD","RDD ID":2} +{"Event":"SparkListenerUnpersistRDD","RDD ID":20} +{"Event":"SparkListenerApplicationEnd","Timestamp":1521483792501} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 88916488c0de..4f8081f2e942 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -29,7 +29,7 @@ import org.mockito.Matchers._ import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -77,7 +77,7 @@ class HeartbeatReceiverSuite heartbeatReceiverClock = new ManualClock heartbeatReceiver = new HeartbeatReceiver(sc, heartbeatReceiverClock) heartbeatReceiverRef = sc.env.rpcEnv.setupEndpoint("heartbeat", heartbeatReceiver) - when(scheduler.executorHeartbeatReceived(any(), any(), any())).thenReturn(true) + when(scheduler.executorHeartbeatReceived(any(), any(), any(), any())).thenReturn(true) } /** @@ -213,8 +213,9 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) + val executorUpdates = new ExecutorMetrics(1234567L, 123456L, 12345L, 1234L, 123L, 12L) val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( - Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId)) + Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId, executorUpdates)) if (executorShouldReregister) { assert(response.reregisterBlockManager) } else { @@ -223,7 +224,8 @@ class HeartbeatReceiverSuite verify(scheduler).executorHeartbeatReceived( Matchers.eq(executorId), Matchers.eq(Array(1L -> metrics.accumulators())), - Matchers.eq(blockManagerId)) + Matchers.eq(blockManagerId), + Matchers.eq(executorUpdates)) } } 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 11b29121739a..3b06126ee9a3 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 @@ -128,6 +128,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "succeeded&failed job list json" -> "applications/local-1422981780767/jobs?status=succeeded&status=failed", "executor list json" -> "applications/local-1422981780767/executors", + "executor list with executor metrics json" -> + "applications/application_1506645932520_24630151/executors", "stage list json" -> "applications/local-1422981780767/stages", "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 2987170bf502..db5ad7f30a04 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -30,6 +30,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} @@ -120,7 +121,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean = true + blockManagerId: BlockManagerId, + executorUpdates: ExecutorMetrics): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -634,7 +636,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi override def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean = true + blockManagerId: BlockManagerId, + executorMetrics: ExecutorMetrics): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None 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 a9e92fa07b9d..59645b2cf181 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -19,7 +19,9 @@ package org.apache.spark.scheduler import java.io.{File, FileOutputStream, InputStream, IOException} +import scala.collection.immutable.Map import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import scala.io.Source import org.apache.hadoop.fs.Path @@ -29,11 +31,14 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} + /** * Test whether EventLoggingListener logs events properly. * @@ -137,6 +142,10 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit "a fine:mind$dollar{bills}.1", None, Some("lz4"))) } + test("Executor metrics update") { + testExecutorMetricsUpdateEventLogging() + } + /* ----------------- * * Actual test logic * * ----------------- */ @@ -251,6 +260,163 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } + /** + * Test executor metrics update logging functionality. This checks that a + * SparkListenerExecutorMetricsUpdate event is added to the Spark history + * log if one of the executor metrics is larger than any previously + * recorded value for the metric, per executor per stage. The task metrics + * should not be added. + */ + private def testExecutorMetricsUpdateEventLogging() { + val conf = getLoggingConf(testDirPath, None) + val logName = "executorMetricsUpdated-test" + val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) + val listenerBus = new LiveListenerBus(conf) + + // list of events and if they should be logged + val events = Array( + (SparkListenerApplicationStart("executionMetrics", None, + 1L, "update", None), true), + (createExecutorAddedEvent(1), true), + (createExecutorAddedEvent(2), true), + (createStageSubmittedEvent(0), true), + (createExecutorMetricsUpdateEvent(1, 10L, 5000L, 50L, 0L, 0L, 0L), true), // new stage + (createExecutorMetricsUpdateEvent(2, 10L, 3500L, 20L, 0L, 0L, 0L), true), // new stage + (createExecutorMetricsUpdateEvent(1, 15L, 4000L, 50L, 0L, 0L, 0L), false), + (createExecutorMetricsUpdateEvent(2, 15L, 3500L, 10L, 0L, 20L, 0L), true), // onheap storage + (createExecutorMetricsUpdateEvent(1, 20L, 6000L, 50L, 0L, 30L, 0L), true), // JVM used + (createExecutorMetricsUpdateEvent(2, 20L, 3500L, 15L, 0L, 20L, 0L), true), // onheap unified + (createStageSubmittedEvent(1), true), + (createExecutorMetricsUpdateEvent(1, 25L, 3000L, 15L, 0L, 0L, 0L), true), // new stage + (createExecutorMetricsUpdateEvent(2, 25L, 6000L, 50L, 0L, 0L, 0L), true), // new stage + (createStageCompletedEvent(0), true), + (createExecutorMetricsUpdateEvent(1, 30L, 3000L, 20L, 0L, 0L, 0L), true), // onheap execution + (createExecutorMetricsUpdateEvent(2, 30L, 5500L, 20L, 0L, 0L, 0L), false), + (createExecutorMetricsUpdateEvent(1, 35L, 3000L, 5L, 25L, 0L, 0L), true), // offheap execution + (createExecutorMetricsUpdateEvent(2, 35L, 5500L, 25L, 0L, 0L, 30L), true), // offheap storage + (createExecutorMetricsUpdateEvent(1, 40L, 3000L, 8L, 20L, 0L, 0L), false), + (createExecutorMetricsUpdateEvent(2, 40L, 5500L, 25L, 0L, 0L, 30L), false), + (createStageCompletedEvent(1), true), + (SparkListenerApplicationEnd(1000L), true)) + + // play the events for the event logger + eventLogger.start() + listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem])) + listenerBus.addToEventLogQueue(eventLogger) + for ((event, included) <- events) { + listenerBus.post(event) + } + listenerBus.stop() + eventLogger.stop() + + // Verify the log file contains the expected events + val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + try { + val lines = readLines(logData) + val logStart = SparkListenerLogStart(SPARK_VERSION) + assert(lines.size === 19) + assert(lines(0).contains("SparkListenerLogStart")) + assert(lines(1).contains("SparkListenerApplicationStart")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) + var i = 1 + for ((event, included) <- events) { + if (included) { + checkEvent(lines(i), event) + i += 1 + } + } + } finally { + logData.close() + } + } + + /** Create a stage submitted event for the specified stage Id. */ + private def createStageSubmittedEvent(stageId: Int) = + SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + + /** Create a stage completed event for the specified stage Id. */ + private def createStageCompletedEvent(stageId: Int) = + SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + + /** 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)) + + /** Create an executor metrics update event, with the specified executor metrics values. */ + private def createExecutorMetricsUpdateEvent( + executorId: Int, time: Long, + jvmUsedMemory: Long, + onHeapExecutionMemory: Long, + offHeapExecutionMemory: Long, + onHeapStorageMemory: Long, + offHeapStorageMemory: Long): SparkListenerExecutorMetricsUpdate = { + val taskMetrics = TaskMetrics.empty + taskMetrics.incDiskBytesSpilled(111) + taskMetrics.incMemoryBytesSpilled(222) + val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) + val executorUpdates = new ExecutorMetrics(time, jvmUsedMemory, onHeapExecutionMemory, + offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory) + SparkListenerExecutorMetricsUpdate( executorId.toString, accum, Some(executorUpdates)) + } + + /** Check that the two ExecutorMetrics match */ + private def checkExecutorMetrics( + executorMetrics1: Option[ExecutorMetrics], + executorMetrics2: Option[ExecutorMetrics]) = { + executorMetrics1 match { + case Some(e1) => + executorMetrics2 match { + case Some(e2) => + assert(e1.timestamp === e2.timestamp) + assert(e1.jvmUsedMemory === e2.jvmUsedMemory) + assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory) + assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory) + assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory) + assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory) + } + case None => + assert(false) + case None => + assert(executorMetrics2.isEmpty) + } + } + + /** Check that the Spark history log line matches the expected event. */ + private def checkEvent(line: String, event: SparkListenerEvent): Unit = { + assert(line.contains(event.getClass.toString.split("\\.").last)) + event match { + case executorMetrics: SparkListenerExecutorMetricsUpdate => + JsonProtocol.sparkEventFromJson(parse(line)) match { + case executorMetrics2: SparkListenerExecutorMetricsUpdate => + assert(executorMetrics.execId === executorMetrics2.execId) + assert(executorMetrics2.accumUpdates.isEmpty) + checkExecutorMetrics(executorMetrics.executorUpdates, executorMetrics2.executorUpdates) + case _ => + assertTypeError("expecting SparkListenerExecutorMetricsUpdate") + } + case stageSubmitted: SparkListenerStageSubmitted => + // accumulables can be different, so only check the stage Id + JsonProtocol.sparkEventFromJson(parse(line)) match { + case logStageSubmitted : SparkListenerStageSubmitted => + assert(logStageSubmitted.stageInfo.stageId == stageSubmitted.stageInfo.stageId) + case _ => + assertTypeError("expecting SparkListenerStageSubmitted") + } + case stageCompleted: SparkListenerStageCompleted => + // accumulables can be different, so only check the stage Id + JsonProtocol.sparkEventFromJson(parse(line)) match { + case logStageSubmitted : SparkListenerStageSubmitted => + assert(logStageSubmitted.stageInfo.stageId == stageCompleted.stageInfo.stageId) + case _ => + assertTypeError("expecting SparkListenerStageCompleted") + } + case _ => + assert(JsonProtocol.sparkEventFromJson(parse(line)) === event) + } + } + private def readLines(in: InputStream): Seq[String] = { Source.fromInputStream(in).getLines().toSeq } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index a4e4ea7cd289..149e58fc5a90 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -89,5 +90,6 @@ private class DummyTaskScheduler extends TaskScheduler { def executorHeartbeatReceived( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], - blockManagerId: BlockManagerId): Boolean = true + blockManagerId: BlockManagerId, + executorMetrics: ExecutorMetrics): Boolean = true } 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 74b72d940eee..e0cc3c46017c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -94,7 +94,8 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates))) + val executorUpdates = Some(new ExecutorMetrics(1234567L, 123456L, 12345L, 1234L, 123L, 12L)) + SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) } val blockUpdated = SparkListenerBlockUpdated(BlockUpdatedInfo(BlockManagerId("Stars", @@ -124,6 +125,7 @@ class JsonProtocolSuite extends SparkFunSuite { testEvent(nodeUnblacklisted, nodeUnblacklistedJsonString) testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) testEvent(blockUpdated, blockUpdatedJsonString) + testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) } test("Dependent Classes") { @@ -419,6 +421,30 @@ class JsonProtocolSuite extends SparkFunSuite { exceptionFailure.accumUpdates, oldExceptionFailure.accumUpdates, (x, y) => x == y) } + test("ExecutorMetricsUpdate backward compatibility: executor metrics update") { + // executorMetricsUpdate was added in 2.1.0. For older event logs, this should + // be set to None. + val executorMetricsUpdate = makeExecutorMetricsUpdate("1", true, true) + val oldExecutorMetricsUpdateJson = + JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) + .removeField( _._1 == "Executor Metrics Updated") + val exepectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", true, false) + assertEquals(exepectedExecutorMetricsUpdate, + JsonProtocol.executorMetricsUpdateFromJson(oldExecutorMetricsUpdateJson)) + } + + test("ExecutorMetricsUpdate: empty metrics update") { + // For SparkListenerExecutorMetricUpdate events, metrics update will be set to + // empty in the event log, to avoid excess logging. + val executorMetricsUpdate = makeExecutorMetricsUpdate("1", true, true) + val oldExecutorMetricsUpdateJson = + JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) + .replace(List("Metrics Updated"), List.empty[JValue]) + val exepectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", false, true) + assertEquals(exepectedExecutorMetricsUpdate, + JsonProtocol.executorMetricsUpdateFromJson(oldExecutorMetricsUpdateJson)) + } + test("AccumulableInfo value de/serialization") { import InternalAccumulator._ val blocks = Seq[(BlockId, BlockStatus)]( @@ -565,6 +591,7 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(stageAttemptId1 === stageAttemptId2) assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) }) + assertEquals(e1.executorUpdates, e2.executorUpdates) case (e1, e2) => assert(e1 === e2) case _ => fail("Events don't match in types!") @@ -654,6 +681,25 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(metrics1.bytesRead === metrics2.bytesRead) } + private def assertEquals(metrics1: Option[ExecutorMetrics], metrics2: Option[ExecutorMetrics]) { + metrics1 match { + case Some(m1) => + metrics2 match { + case Some(m2) => + assert(m1.timestamp === m2.timestamp) + assert(m1.jvmUsedMemory === m2.jvmUsedMemory) + assert(m1.onHeapExecutionMemory === m2.onHeapExecutionMemory) + assert(m1.offHeapExecutionMemory === m2.offHeapExecutionMemory) + assert(m1.onHeapStorageMemory === m2.onHeapStorageMemory) + assert(m1.offHeapStorageMemory === m2.offHeapStorageMemory) + case None => + assert(false) + } + case None => + assert(metrics2.isEmpty) + } + } + private def assertEquals(result1: JobResult, result2: JobResult) { (result1, result2) match { case (JobSucceeded, JobSucceeded) => @@ -820,6 +866,26 @@ private[spark] object JsonProtocolSuite extends Assertions { new AccumulableInfo(id, Some(s"Accumulable$id"), Some(s"delta$id"), Some(s"val$id"), internal, countFailedValues, metadata) + /** Creates an SparkListenerExecutorMetricsUpdate event */ + private def makeExecutorMetricsUpdate(execId: String, + includeTaskMetrics: Boolean, + includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { + val taskMetrics = + if (includeTaskMetrics) { + Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), + makeAccumulableInfo(2, false, false, None)))) + } else { + Seq() + } + val executorMetricsUpdate = + if (includeExecutorMetrics) { + Some(new ExecutorMetrics(1234567L, 123456L, 0L, 0L, 0L, 0L)) + } else { + None + } + SparkListenerExecutorMetricsUpdate(execId, taskMetrics, executorMetricsUpdate) + } + /** * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is * set to true) or read data from a shuffle otherwise. @@ -2007,7 +2073,16 @@ private[spark] object JsonProtocolSuite extends Assertions { | } | ] | } - | ] + | ], + | "Executor Metrics Updated" : { + | "Timestamp" : 1234567, + | "JVM Used Memory" : 123456, + | "Onheap Execution Memory" : 12345, + | "Offheap Execution Memory" : 1234, + | "Onheap Storage Memory" : 123, + | "Offheap Storage Memory" : 12 + | } + | |} """.stripMargin diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 9552d001a079..698921456109 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -77,6 +77,7 @@ app-20180109111548-0000 app-20161115172038-0000 app-20161116163331-0000 application_1516285256255_0012 +application_1506645932520_24630151 local-1422981759269 local-1422981780767 local-1425081759269 From da83f2e58ff7d495111a0c1f36bf54ebcf35d444 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Sun, 1 Apr 2018 19:13:41 -0700 Subject: [PATCH 10/29] modify MimaExcludes.scala to filter changes to SparkListenerExecutorMetricsUpdate --- project/MimaExcludes.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4f6d5ff89868..ebca8c85deec 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -89,7 +89,13 @@ object MimaExcludes { ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol") + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), + + // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate$") ) // Exclude rules for 2.3.x From f25a44b95e4e6a8532c6541ee985789dff5bc7de Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Sat, 21 Apr 2018 17:02:57 -0700 Subject: [PATCH 11/29] Address code review comments, change event logging to stage end. --- .../org/apache/spark/executor/Executor.scala | 56 +++- .../spark/executor/ExecutorMetrics.scala | 19 +- .../apache/spark/scheduler/DAGScheduler.scala | 16 +- .../scheduler/EventLoggingListener.scala | 47 ++-- .../spark/scheduler/PeakExecutorMetrics.scala | 105 ++++--- .../org/apache/spark/status/LiveEntity.scala | 2 +- .../org/apache/spark/status/api/v1/api.scala | 7 +- .../org/apache/spark/util/JsonProtocol.scala | 22 +- .../application_list_json_expectation.json | 10 +- .../completed_app_list_json_expectation.json | 10 +- ...ith_executor_metrics_json_expectation.json | 259 +++++++----------- .../limit_app_list_json_expectation.json | 10 +- .../minDate_app_list_json_expectation.json | 10 +- .../minEndDate_app_list_json_expectation.json | 10 +- .../application_1506645932520_24630151 | 163 ++++------- .../apache/spark/HeartbeatReceiverSuite.scala | 3 +- .../scheduler/EventLoggingListenerSuite.scala | 189 +++++++++---- .../spark/status/AppStatusListenerSuite.scala | 106 ++++++- .../apache/spark/util/JsonProtocolSuite.scala | 48 ++-- project/MimaExcludes.scala | 3 + 20 files changed, 644 insertions(+), 451 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 14966cee909a..7cabeee14d43 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -19,12 +19,13 @@ package org.apache.spark.executor import java.io.{File, NotSerializableException} import java.lang.Thread.UncaughtExceptionHandler -import java.lang.management.ManagementFactory +import java.lang.management.{BufferPoolMXBean, ManagementFactory} import java.net.{URI, URL} import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent._ import javax.annotation.concurrent.GuardedBy +import javax.management.ObjectName import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -36,7 +37,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} +import org.apache.spark.memory.{MemoryManager, SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription} import org.apache.spark.shuffle.FetchFailedException @@ -71,6 +72,12 @@ private[spark] class Executor( private val conf = env.conf + // BufferPoolMXBean for direct memory + private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME) + + // BufferPoolMXBean for mapped memory + private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME) + // No ip or host:port - just hostname Utils.checkHost(executorHostname) // must not have port specified. @@ -788,10 +795,8 @@ private[spark] class Executor( val curGCTime = computeTotalGcTime() // get executor level memory metrics - val executorUpdates = new ExecutorMetrics(System.currentTimeMillis(), - ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), - env.memoryManager.onHeapExecutionMemoryUsed, env.memoryManager.offHeapExecutionMemoryUsed, - env.memoryManager.onHeapStorageMemoryUsed, env.memoryManager.offHeapStorageMemoryUsed) + val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager, + directBufferPool, mappedBufferPool) for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { @@ -829,4 +834,43 @@ private[spark] object Executor { // task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be // used instead. val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties] + + val DIRECT_BUFFER_POOL_NAME = "direct" + val MAPPED_BUFFER_POOL_NAME = "mapped" + + /** Get the BufferPoolMXBean for the specified buffer pool. */ + def getBufferPool(pool: String): BufferPoolMXBean = { + val name = new ObjectName("java.nio:type=BufferPool,name=" + pool) + ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, + name.toString, classOf[BufferPoolMXBean]) + } + + /** + * Get the current executor level memory metrics. + * + * @param memoryManager the memory manager + * @param direct the direct memory buffer pool + * @param mapped the mapped memory buffer pool + * @return the executor memory metrics + */ + def getCurrentExecutorMetrics( + memoryManager: MemoryManager, + direct: BufferPoolMXBean, + mapped: BufferPoolMXBean) : ExecutorMetrics = { + val onHeapExecutionMemoryUsed = memoryManager.onHeapExecutionMemoryUsed + val offHeapExecutionMemoryUsed = memoryManager.offHeapExecutionMemoryUsed + val onHeapStorageMemoryUsed = memoryManager.onHeapStorageMemoryUsed + val offHeapStorageMemoryUsed = memoryManager.offHeapStorageMemoryUsed + new ExecutorMetrics(System.currentTimeMillis(), + ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), + ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed(), + onHeapExecutionMemoryUsed, + offHeapExecutionMemoryUsed, + onHeapStorageMemoryUsed, + offHeapStorageMemoryUsed, + onHeapExecutionMemoryUsed + onHeapStorageMemoryUsed, // on heap unified memory + offHeapExecutionMemoryUsed + offHeapStorageMemoryUsed, // off heap unified memory + direct.getMemoryUsed, + mapped.getMemoryUsed) + } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 906bdc41d05b..6d35a1c682e2 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -26,18 +26,29 @@ import org.apache.spark.annotation.DeveloperApi * This is sent to the driver periodically (on executor heartbeat), to provide * information about each executor's metrics. * - * @param timestamp the time the metrics were collected - * @param jvmUsedMemory the amount of JVM used memory for the executor + * @param timestamp the time the metrics were collected, or -1 for Spark history + * log events which are logged when a stage has completed + * @param jvmUsedHeapMemory the amount of JVM used heap memory for the executor + * @param jvmUsedNonHeapMemory the amount of JVM used non-heap memory for the executor * @param onHeapExecutionMemory the amount of on heap execution memory used * @param offHeapExecutionMemory the amount of off heap execution memory used * @param onHeapStorageMemory the amount of on heap storage memory used * @param offHeapStorageMemory the amount of off heap storage memory used + * @param onHeapUnifiedMemory the amount of on heap unified region memory used + * @param offHeapUnifiedMemory the amount of off heap unified region memory used + * @param directMemory the amount of direct memory used + * @param mappedMemory the amount of mapped memory used */ @DeveloperApi class ExecutorMetrics private[spark] ( val timestamp: Long, - val jvmUsedMemory: Long, + val jvmUsedHeapMemory: Long, + val jvmUsedNonHeapMemory: Long, val onHeapExecutionMemory: Long, val offHeapExecutionMemory: Long, val onHeapStorageMemory: Long, - val offHeapStorageMemory: Long) extends Serializable + val offHeapStorageMemory: Long, + val onHeapUnifiedMemory: Long, + val offHeapUnifiedMemory: Long, + val directMemory: Long, + val mappedMemory: Long) extends Serializable diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4f3a0731db3b..762c7d0bd719 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -35,7 +35,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.{Executor, ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils @@ -214,6 +214,12 @@ class DAGScheduler( private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + /** BufferPoolMXBean for direct memory */ + private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME) + + /** BufferPoolMXBean for mapped memory */ + private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME) + /** * Called by the TaskSetManager to report task's starting. */ @@ -1764,12 +1770,8 @@ class DAGScheduler( /** Reports heartbeat metrics for the driver. */ private def reportHeartBeat(): Unit = { // get driver memory metrics - val driverUpdates = new ExecutorMetrics(System.currentTimeMillis(), - ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), - sc.env.memoryManager.onHeapExecutionMemoryUsed, - sc.env.memoryManager.offHeapExecutionMemoryUsed, - sc.env.memoryManager.onHeapStorageMemoryUsed, - sc.env.memoryManager.offHeapStorageMemoryUsed) + val driverUpdates = Executor.getCurrentExecutorMetrics( + sc.env.memoryManager, directBufferPool, mappedBufferPool) val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, Some(driverUpdates))) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index e159421f4fbb..5fae14bb1708 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -94,8 +94,9 @@ private[spark] class EventLoggingListener( // Visible for tests only. private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) - // Peak metric values for each executor - private var peakExecutorMetrics = new mutable.HashMap[String, PeakExecutorMetrics]() + // map of live stages, to peak executor metrics for the stage + private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int), + mutable.HashMap[String, PeakExecutorMetrics]]() /** * Creates the log file in the configured log directory. @@ -162,7 +163,8 @@ private[spark] class EventLoggingListener( override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { logEvent(event) // clear the peak metrics when a new stage starts - peakExecutorMetrics.values.foreach(_.reset()) + liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), + new mutable.HashMap[String, PeakExecutorMetrics]()) } override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) @@ -177,6 +179,27 @@ private[spark] class EventLoggingListener( // Events that trigger a flush override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { + // log the peak executor metrics for the stage, for each executor + val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() + val executorMap = liveStageExecutorMetrics.remove( + (event.stageInfo.stageId, event.stageInfo.attemptNumber())) + executorMap.foreach { + executorEntry => { + for ((executorId, peakExecutorMetrics) <- executorEntry) { + val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory, + peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory, + peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory, + peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory, + peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory, + peakExecutorMetrics.mappedMemory) + val executorUpdate = new SparkListenerExecutorMetricsUpdate( + executorId, accumUpdates, Some(executorMetrics)) + logEvent(executorUpdate) + } + } + } + + // log stage completed event logEvent(event, flushLogger = true) } @@ -205,12 +228,10 @@ private[spark] class EventLoggingListener( } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { logEvent(event, flushLogger = true) - peakExecutorMetrics.put(event.executorId, new PeakExecutorMetrics()) } override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { logEvent(event, flushLogger = true) - peakExecutorMetrics.remove(event.executorId) } override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { @@ -244,19 +265,13 @@ private[spark] class EventLoggingListener( } } - /** - * Log if there is a new peak value for one of the memory metrics for the given executor. - * Metrics are cleared out when a new stage is started in onStageSubmitted, so this will - * log new peak memory metric values per executor per stage. - */ override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - var log: Boolean = false + // For the active stages, record any new peak values for the memory metrics for the executor event.executorUpdates.foreach { executorUpdates => - val peakMetrics = peakExecutorMetrics.getOrElseUpdate(event.execId, new PeakExecutorMetrics()) - if (peakMetrics.compareAndUpdate(executorUpdates)) { - val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() - logEvent(new SparkListenerExecutorMetricsUpdate(event.execId, accumUpdates, - event.executorUpdates), flushLogger = true) + liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => + val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + event.execId, new PeakExecutorMetrics()) + peakMetrics.compareAndUpdate(executorUpdates) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index 187cc77751e6..d554c938f5e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -21,17 +21,40 @@ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.status.api.v1.PeakMemoryMetrics /** - * Records the peak values for executor level metrics. If jvmUsedMemory is -1, then no values have - * been recorded yet. + * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no + * values have been recorded yet. */ private[spark] class PeakExecutorMetrics { - private var jvmUsedMemory = -1L; - private var onHeapExecutionMemory = 0L - private var offHeapExecutionMemory = 0L - private var onHeapStorageMemory = 0L - private var offHeapStorageMemory = 0L - private var onHeapUnifiedMemory = 0L - private var offHeapUnifiedMemory = 0L + private var _jvmUsedHeapMemory = -1L; + private var _jvmUsedNonHeapMemory = 0L; + private var _onHeapExecutionMemory = 0L + private var _offHeapExecutionMemory = 0L + private var _onHeapStorageMemory = 0L + private var _offHeapStorageMemory = 0L + private var _onHeapUnifiedMemory = 0L + private var _offHeapUnifiedMemory = 0L + private var _directMemory = 0L + private var _mappedMemory = 0L + + def jvmUsedHeapMemory: Long = _jvmUsedHeapMemory + + def jvmUsedNonHeapMemory: Long = _jvmUsedNonHeapMemory + + def onHeapExecutionMemory: Long = _onHeapExecutionMemory + + def offHeapExecutionMemory: Long = _offHeapExecutionMemory + + def onHeapStorageMemory: Long = _onHeapStorageMemory + + def offHeapStorageMemory: Long = _offHeapStorageMemory + + def onHeapUnifiedMemory: Long = _onHeapUnifiedMemory + + def offHeapUnifiedMemory: Long = _offHeapUnifiedMemory + + def directMemory: Long = _directMemory + + def mappedMemory: Long = _mappedMemory /** * Compare the specified memory values with the saved peak executor memory @@ -43,36 +66,44 @@ private[spark] class PeakExecutorMetrics { def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { var updated: Boolean = false - if (executorMetrics.jvmUsedMemory > jvmUsedMemory) { - jvmUsedMemory = executorMetrics.jvmUsedMemory + if (executorMetrics.jvmUsedHeapMemory > _jvmUsedHeapMemory) { + _jvmUsedHeapMemory = executorMetrics.jvmUsedHeapMemory + updated = true + } + if (executorMetrics.jvmUsedNonHeapMemory > _jvmUsedNonHeapMemory) { + _jvmUsedNonHeapMemory = executorMetrics.jvmUsedNonHeapMemory + updated = true + } + if (executorMetrics.onHeapExecutionMemory > _onHeapExecutionMemory) { + _onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory updated = true } - if (executorMetrics.onHeapExecutionMemory > onHeapExecutionMemory) { - onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory + if (executorMetrics.offHeapExecutionMemory > _offHeapExecutionMemory) { + _offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory updated = true } - if (executorMetrics.offHeapExecutionMemory > offHeapExecutionMemory) { - offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory + if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) { + _onHeapStorageMemory = executorMetrics.onHeapStorageMemory updated = true } - if (executorMetrics.onHeapStorageMemory > onHeapStorageMemory) { - onHeapStorageMemory = executorMetrics.onHeapStorageMemory + if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) { + _offHeapStorageMemory = executorMetrics.offHeapStorageMemory updated = true } - if (executorMetrics.offHeapStorageMemory > offHeapStorageMemory) { - offHeapStorageMemory = executorMetrics.offHeapStorageMemory + if (executorMetrics.onHeapUnifiedMemory > _onHeapUnifiedMemory) { + _onHeapUnifiedMemory = executorMetrics.onHeapUnifiedMemory updated = true } - val newOnHeapUnifiedMemory = (executorMetrics.onHeapExecutionMemory + - executorMetrics.onHeapStorageMemory) - if (newOnHeapUnifiedMemory > onHeapUnifiedMemory) { - onHeapUnifiedMemory = newOnHeapUnifiedMemory + if (executorMetrics.offHeapUnifiedMemory > _offHeapUnifiedMemory) { + _offHeapUnifiedMemory = executorMetrics.offHeapUnifiedMemory updated = true } - val newOffHeapUnifiedMemory = (executorMetrics.offHeapExecutionMemory + - executorMetrics.offHeapStorageMemory) - if ( newOffHeapUnifiedMemory > offHeapUnifiedMemory) { - offHeapUnifiedMemory = newOffHeapUnifiedMemory + if (executorMetrics.directMemory > _directMemory) { + _directMemory = executorMetrics.directMemory + updated = true + } + if (executorMetrics.mappedMemory > _mappedMemory) { + _mappedMemory = executorMetrics.mappedMemory updated = true } @@ -84,23 +115,13 @@ private[spark] class PeakExecutorMetrics { * values set. */ def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = { - if (jvmUsedMemory < 0) { + if (_jvmUsedHeapMemory < 0) { None } else { - Some(new PeakMemoryMetrics(jvmUsedMemory, onHeapExecutionMemory, - offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory, - onHeapUnifiedMemory, offHeapUnifiedMemory)) + Some(new PeakMemoryMetrics(_jvmUsedHeapMemory, _jvmUsedNonHeapMemory, + _onHeapExecutionMemory, _offHeapExecutionMemory, _onHeapStorageMemory, + _offHeapStorageMemory, _onHeapUnifiedMemory, _offHeapUnifiedMemory, + _directMemory, _mappedMemory)) } } - - /** Clears/resets the saved peak values. */ - def reset(): Unit = { - jvmUsedMemory = -1L; - onHeapExecutionMemory = 0L - offHeapExecutionMemory = 0L - onHeapStorageMemory = 0L - offHeapStorageMemory = 0L - onHeapUnifiedMemory = 0L - offHeapUnifiedMemory = 0L - } } 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 afb3e60e0b28..c296f55210f2 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -269,7 +269,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE def hasMemoryInfo: Boolean = totalOnHeap >= 0L // peak values for executor level metrics - var peakExecutorMetrics = new PeakExecutorMetrics + val peakExecutorMetrics = new PeakExecutorMetrics def hostname: String = if (host != null) host else hostPort.split(":")(0) 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 4d14ba7560e8..15f83213bb6e 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 @@ -109,13 +109,16 @@ class MemoryMetrics private[spark]( val totalOffHeapStorageMemory: Long) class PeakMemoryMetrics private[spark]( - val jvmUsedMemory: Long, + val jvmUsedHeapMemory: Long, + val jvmUsedNonHeapMemory: Long, val onHeapExecutionMemory: Long, val offHeapExecutionMemory: Long, val onHeapStorageMemory: Long, val offHeapStorageMemory: Long, val onHeapUnifiedMemory: Long, - val offHeapUnifiedMemory: Long) + val offHeapUnifiedMemory: Long, + val directMemory: Long, + val mappedMemory: Long) class JobData private[spark]( val jobId: Int, 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 c2625efbcf55..8a36a93c858e 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -390,11 +390,16 @@ private[spark] object JsonProtocol { */ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { ("Timestamp" -> executorMetrics.timestamp) ~ - ("JVM Used Memory" -> executorMetrics.jvmUsedMemory) ~ + ("JVM Used Heap Memory" -> executorMetrics.jvmUsedHeapMemory) ~ + ("JVM Used Nonheap Memory" -> executorMetrics.jvmUsedNonHeapMemory) ~ ("Onheap Execution Memory" -> executorMetrics.onHeapExecutionMemory) ~ ("Offheap Execution Memory" -> executorMetrics.offHeapExecutionMemory) ~ ("Onheap Storage Memory" -> executorMetrics.onHeapStorageMemory) ~ - ("Offheap Storage Memory" -> executorMetrics.offHeapStorageMemory) + ("Offheap Storage Memory" -> executorMetrics.offHeapStorageMemory) ~ + ("Onheap Unified Memory" -> executorMetrics.onHeapUnifiedMemory) ~ + ("Offheap Unified Memory" -> executorMetrics.offHeapUnifiedMemory) ~ + ("Direct Memory" -> executorMetrics.directMemory) ~ + ("Mapped Memory" -> executorMetrics.mappedMemory) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -611,13 +616,20 @@ private[spark] object JsonProtocol { */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val timeStamp = (json \ "Timestamp").extract[Long] - val jvmUsedMemory = (json \ "JVM Used Memory").extract[Long] + val jvmUsedHeapMemory = (json \ "JVM Used Heap Memory").extract[Long] + val jvmUsedNonHeapMemory = (json \ "JVM Used Nonheap Memory").extract[Long] val onHeapExecutionMemory = (json \ "Onheap Execution Memory").extract[Long] val offHeapExecutionMemory = (json \ "Offheap Execution Memory").extract[Long] val onHeapStorageMemory = (json \ "Onheap Storage Memory").extract[Long] val offHeapStorageMemory = (json \ "Offheap Storage Memory").extract[Long] - new ExecutorMetrics(timeStamp, jvmUsedMemory, onHeapExecutionMemory, - offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory) + val onHeapUnifiedMemory = (json \ "Onheap Unified Memory").extract[Long] + val offHeapUnifiedMemory = (json \ "Offheap Unified Memory").extract[Long] + val directMemory = (json \ "Direct Memory").extract[Long] + val mappedMemory = (json \ "Mapped Memory").extract[Long] + new ExecutorMetrics(timeStamp, jvmUsedHeapMemory, jvmUsedNonHeapMemory, + onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, + offHeapStorageMemory, onHeapUnifiedMemory, offHeapUnifiedMemory, directMemory, + mappedMemory) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { 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 9858c73fae2f..eea6f595efd2 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -2,16 +2,16 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501 + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134 } ] }, { "id" : "application_1516285256255_0012", 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 94eddfa0c7bb..7bc7f31be097 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 @@ -2,15 +2,15 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501, + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134, "lastUpdatedEpoch" : 0 } ] }, { 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 1cd5fbb794cb..ac1bb97b157f 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 @@ -1,6 +1,6 @@ [ { "id" : "driver", - "hostPort" : "cluster-node0033.company.com:53121", + "hostPort" : "node0033.grid.company.com:60749", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -17,28 +17,31 @@ "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : false, - "maxMemory" : 1046269132, - "addTime" : "2018-03-19T18:21:24.751GMT", + "maxMemory" : 1043437977, + "addTime" : "2018-04-19T23:55:05.107GMT", "executorLogs" : { }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 1046269132, + "totalOnHeapStorageMemory" : 1043437977, "totalOffHeapStorageMemory" : 0 }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 671982856, + "jvmUsedHeapMemory" : 629553808, + "jvmUsedNonHeapMemory" : 205304696, "onHeapExecutionMemory" : 0, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 900126, + "onHeapStorageMemory" : 905801, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 900126, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 905801, + "offHeapUnifiedMemory" : 0, + "directMemory" : 397602, + "mappedMemory" : 0 } }, { - "id" : "8", - "hostPort" : "cluster-node6128.company.com:1664", + "id" : "7", + "hostPort" : "node6340.grid.company.com:5933", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -56,51 +59,10 @@ "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:22:25.988GMT", - "executorLogs" : { - "stdout" : "http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 956615884, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "jvmUsedMemory" : 226589176, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 0, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 0, - "offHeapUnifiedMemory" : 0 - } -}, { - "id" : "7", - "hostPort" : "cluster-node2449.company.com:26914", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 1, - "maxTasks" : 1, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 11073, - "totalGCTime" : 942, - "totalInputBytes" : 36849246, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 349006, - "isBlacklisted" : false, - "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:22:10.182GMT", + "addTime" : "2018-04-19T23:55:49.826GMT", "executorLogs" : { - "stdout" : "http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stderr?start=-4096" + "stdout" : "http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096", + "stderr" : "http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -108,19 +70,10 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "jvmUsedMemory" : 365803960, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 562630, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 562630, - "offHeapUnifiedMemory" : 0 - } + "blacklistedInStages" : [ ] }, { "id" : "6", - "hostPort" : "cluster-node6148.company.com:1233", + "hostPort" : "node6644.grid.company.com:8445", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -129,19 +82,19 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 34956, - "totalGCTime" : 5484, - "totalInputBytes" : 24250210, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, "totalShuffleRead" : 0, - "totalShuffleWrite" : 242714, + "totalShuffleWrite" : 0, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:22:19.537GMT", + "addTime" : "2018-04-19T23:55:47.549GMT", "executorLogs" : { - "stdout" : "http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stderr?start=-4096" + "stdout" : "http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stdout?start=-4096", + "stderr" : "http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -149,19 +102,10 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "jvmUsedMemory" : 501950360, - "onHeapExecutionMemory" : 35651584, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 562630, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 36214214, - "offHeapUnifiedMemory" : 0 - } + "blacklistedInStages" : [ ] }, { "id" : "5", - "hostPort" : "cluster-node5882.company.com:24590", + "hostPort" : "node2477.grid.company.com:20123", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -172,17 +116,17 @@ "failedTasks" : 0, "completedTasks" : 1, "totalTasks" : 1, - "totalDuration" : 30945, - "totalGCTime" : 6111, + "totalDuration" : 9252, + "totalGCTime" : 920, "totalInputBytes" : 36838295, "totalShuffleRead" : 0, "totalShuffleWrite" : 355051, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:22:11.267GMT", + "addTime" : "2018-04-19T23:55:43.160GMT", "executorLogs" : { - "stdout" : "http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stderr?start=-4096" + "stdout" : "http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stdout?start=-4096", + "stderr" : "http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -190,19 +134,10 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "jvmUsedMemory" : 513138800, - "onHeapExecutionMemory" : 41943040, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 562630, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 42505670, - "offHeapUnifiedMemory" : 0 - } + "blacklistedInStages" : [ ] }, { "id" : "4", - "hostPort" : "cluster-node6137.company.com:19036", + "hostPort" : "node4243.grid.company.com:16084", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -211,19 +146,19 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 79365, - "totalGCTime" : 6979, - "totalInputBytes" : 50409514, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 31362123, + "completedTasks" : 3, + "totalTasks" : 3, + "totalDuration" : 15645, + "totalGCTime" : 405, + "totalInputBytes" : 87272855, + "totalShuffleRead" : 438675, + "totalShuffleWrite" : 26773039, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:21:44.950GMT", + "addTime" : "2018-04-19T23:55:12.278GMT", "executorLogs" : { - "stdout" : "http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stderr?start=-4096" + "stdout" : "http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stdout?start=-4096", + "stderr" : "http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -233,17 +168,20 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 707255176, - "onHeapExecutionMemory" : 0, + "jvmUsedHeapMemory" : 518613056, + "jvmUsedNonHeapMemory" : 95657456, + "onHeapExecutionMemory" : 37748736, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 83580552, + "onHeapStorageMemory" : 63104457, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 83580552, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 100853193, + "offHeapUnifiedMemory" : 0, + "directMemory" : 126261, + "mappedMemory" : 0 } }, { "id" : "3", - "hostPort" : "cluster-node1177.company.com:12614", + "hostPort" : "node0998.grid.company.com:45265", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -252,19 +190,19 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 2, - "totalTasks" : 2, - "totalDuration" : 38198, - "totalGCTime" : 1024, - "totalInputBytes" : 50423609, - "totalShuffleRead" : 438675, - "totalShuffleWrite" : 26424033, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 14491, + "totalGCTime" : 342, + "totalInputBytes" : 50409514, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 31362123, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:21:33.370GMT", + "addTime" : "2018-04-19T23:55:12.088GMT", "executorLogs" : { - "stdout" : "http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stderr?start=-4096" + "stdout" : "http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stdout?start=-4096", + "stderr" : "http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -274,17 +212,20 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 620936920, + "jvmUsedHeapMemory" : 726805712, + "jvmUsedNonHeapMemory" : 90709624, "onHeapExecutionMemory" : 0, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 72624976, + "onHeapStorageMemory" : 69535048, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 72624976, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 69535048, + "offHeapUnifiedMemory" : 0, + "directMemory" : 87796, + "mappedMemory" : 0 } }, { "id" : "2", - "hostPort" : "cluster-node6036.company.com:17435", + "hostPort" : "node4045.grid.company.com:29262", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -295,17 +236,17 @@ "failedTasks" : 0, "completedTasks" : 1, "totalTasks" : 1, - "totalDuration" : 44061, - "totalGCTime" : 4677, + "totalDuration" : 14113, + "totalGCTime" : 326, "totalInputBytes" : 50423423, "totalShuffleRead" : 0, "totalShuffleWrite" : 22950296, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:21:41.213GMT", + "addTime" : "2018-04-19T23:55:12.471GMT", "executorLogs" : { - "stdout" : "http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stderr?start=-4096" + "stdout" : "http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stdout?start=-4096", + "stderr" : "http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -315,17 +256,20 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 695372944, + "jvmUsedHeapMemory" : 595946552, + "jvmUsedNonHeapMemory" : 91208368, "onHeapExecutionMemory" : 0, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 58468552, + "onHeapStorageMemory" : 58468944, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 58468552, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 58468944, + "offHeapUnifiedMemory" : 0, + "directMemory" : 87796, + "mappedMemory" : 0 } }, { "id" : "1", - "hostPort" : "cluster-node4342.company.com:29144", + "hostPort" : "node1404.grid.company.com:34043", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -334,19 +278,19 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 2, - "totalTasks" : 2, - "totalDuration" : 31501, - "totalGCTime" : 936, - "totalInputBytes" : 74654808, + "completedTasks" : 3, + "totalTasks" : 3, + "totalDuration" : 15665, + "totalGCTime" : 471, + "totalInputBytes" : 98905018, "totalShuffleRead" : 0, - "totalShuffleWrite" : 20352030, + "totalShuffleWrite" : 20594744, "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-03-19T18:21:33.307GMT", + "addTime" : "2018-04-19T23:55:11.695GMT", "executorLogs" : { - "stdout" : "http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stdout?start=-4096", - "stderr" : "http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stderr?start=-4096" + "stdout" : "http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stdout?start=-4096", + "stderr" : "http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, @@ -356,12 +300,15 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedMemory" : 505317000, - "onHeapExecutionMemory" : 35651584, + "jvmUsedHeapMemory" : 755008624, + "jvmUsedNonHeapMemory" : 100519936, + "onHeapExecutionMemory" : 0, "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 47961406, + "onHeapStorageMemory" : 47962185, "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 83612990, - "offHeapUnifiedMemory" : 0 + "onHeapUnifiedMemory" : 47962185, + "offHeapUnifiedMemory" : 0, + "directMemory" : 98230, + "mappedMemory" : 0 } } ] 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 bffe37ce52b4..9e1e65a35881 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 @@ -2,16 +2,16 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501 + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134 } ] }, { "id" : "application_1516285256255_0012", 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 066217dacbe9..28c6bf1b3e01 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 @@ -2,16 +2,16 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501 + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134 } ] }, { "id" : "application_1516285256255_0012", 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 b1226c905d6a..f547b79f47e1 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 @@ -2,16 +2,16 @@ "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-03-19T18:21:00.843GMT", - "endTime" : "2018-03-19T18:23:12.501GMT", + "startTime" : "2018-04-19T23:54:42.734GMT", + "endTime" : "2018-04-19T23:56:29.134GMT", "lastUpdated" : "", - "duration" : 131658, + "duration" : 106400, "sparkUser" : "edlu", "completed" : true, "appSparkVersion" : "2.4.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1521483660843, - "endTimeEpoch" : 1521483792501 + "startTimeEpoch" : 1524182082734, + "endTimeEpoch" : 1524182189134 } ] }, { "id" : "application_1516285256255_0012", diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 index 26289113d369..b3424f9582c0 100644 --- a/core/src/test/resources/spark-events/application_1506645932520_24630151 +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -1,106 +1,63 @@ {"Event":"SparkListenerLogStart","Spark Version":"2.4.0-SNAPSHOT"} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483677960,"JVM Used Memory":587908264,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"cluster-node0033.company.com","Port":53121},"Maximum Memory":1046269132,"Timestamp":1521483684751,"Maximum Onheap Memory":1046269132,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_31/jre","Java Version":"1.8.0_31 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.jars.ivySettings":"/export/apps/spark/commonconf/ivysettings.xml","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.driver.host":"cluster-node0033.company.com","spark.dynamicAllocation.sustainedSchedulerBacklogTimeout":"5","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"36653","spark.shuffle.service.enabled":"true","spark.ui.acls.enable":"true","spark.reducer.maxSizeInFlight":"48m","spark.yarn.queue":"spark_default","spark.repl.class.uri":"spark://cluster-node0033.company.com:36653/classes","spark.jars":"","spark.yarn.historyServer.address":"clustersh01.company.com:18080","spark.memoryOverhead.multiplier.percent":"10","spark.repl.class.outputDir":"/grid/a/mapred/tmp/spark-c015ad60-cb0b-4920-a14d-e79aa91000f3/repl-c23128a3-a417-40f7-933e-3cd0a5e7c747","spark.dynamicAllocation.cachedExecutorIdleTimeout":"1200","spark.yarn.access.namenodes":"hdfs://clusternn02.company.com:9000","spark.app.name":"Spark shell","spark.dynamicAllocation.schedulerBacklogTimeout":"5","spark.yarn.security.credentials.hive.enabled":"false","spark.yarn.am.cores":"1","spark.memoryOverhead.min":"384","spark.scheduler.mode":"FIFO","spark.driver.memory":"2G","spark.executor.instances":"4","spark.isolated.classloader.additional.classes.prefix":"com_company_","spark.logConf":"true","spark.ui.showConsoleProgress":"true","spark.user.priority.jars":"*********(redacted)","spark.isolated.classloader":"true","spark.sql.sources.schemaStringLengthThreshold":"40000","spark.yarn.secondary.jars":"spark-avro_2.11-3.2.0.21.jar,grid-topology-1.0.jar","spark.reducer.maxBlocksInFlightPerAddress":"100","spark.dynamicAllocation.maxExecutors":"900","spark.yarn.appMasterEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.executor.id":"driver","spark.yarn.am.memory":"2G","spark.driver.cores":"1","spark.min.mem.vore.ratio":"5","spark.sql.sources.partitionOverwriteMode":"DYNAMIC","spark.submit.deployMode":"client","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.isolated.classloader.default.jar":"*dali-data-spark*","spark.authenticate":"true","spark.eventLog.usexattr":"true","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.memory":"2G","spark.home":"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51","spark.reducer.maxReqsInFlight":"10","spark.eventLog.dir":"hdfs://clusternn02.company.com:9000/system/spark-history","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.isolated.classes":"org.apache.hadoop.hive.ql.io.CombineHiveInputFormat$CombineHiveInputSplit","spark.eventLog.compress":"true","spark.executor.cores":"1","spark.version":"2.1.0","spark.driver.appUIAddress":"http://cluster-node0033.company.com:46527","spark.repl.local.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"clusterwp01.company.com","spark.min.memory-gb.size":"10","spark.dynamicAllocation.minExecutors":"1","spark.dynamicAllocation.initialExecutors":"3","spark.expressionencoder.org.apache.avro.specific.SpecificRecord":"com.databricks.spark.avro.AvroEncoder$","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://clusterwp01.company.com:8080/proxy/application_1506645932520_24630151","spark.executorEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.dynamicAllocation.executorIdleTimeout":"150","spark.shell.auto.node.labeling":"true","spark.yarn.dist.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.app.id":"application_1506645932520_24630151","spark.ui.view.acls":"*"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_31/jre/lib/amd64","user.dir":"*********(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.31-b07","java.endorsed.dirs":"/usr/java/jdk1.8.0_31/jre/lib/endorsed","java.runtime.version":"1.8.0_31-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_31/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_31/jre/lib/resources.jar:/usr/java/jdk1.8.0_31/jre/lib/rt.jar:/usr/java/jdk1.8.0_31/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_31/jre/lib/jsse.jar:/usr/java/jdk1.8.0_31/jre/lib/jce.jar:/usr/java/jdk1.8.0_31/jre/lib/charsets.jar:/usr/java/jdk1.8.0_31/jre/lib/jfr.jar:/usr/java/jdk1.8.0_31/jre/classes","file.encoding":"ANSI_X3.4-1968","user.timezone":"*********(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"2.6.32-504.16.2.el6.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*********(redacted)","sun.jnu.encoding":"ANSI_X3.4-1968","user.language":"*********(redacted)","java.vendor.url":"*********(redacted)","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":"*********(redacted)","user.name":"*********(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --class org.apache.spark.repl.Main --name Spark shell --jars /export/home/edlu/spark-avro_2.11-3.2.0.21.jar,/export/apps/hadoop/site/lib/grid-topology-1.0.jar --num-executors 4 spark-shell","java.home":"/usr/java/jdk1.8.0_31/jre","java.version":"1.8.0_31","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-servlet-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/derby-10.12.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-reflect-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-graphx_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-util-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/base64-2.3.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-auth-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/validation-api-1.1.0.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-api-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/objenesis-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/conf/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpclient-4.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/kryo-shaded-3.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-library-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-net-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xz-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-annotations-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/activation-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arpack_combined_all-0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libthrift-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aircompressor-0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-jackson-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/asm-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-hive_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ivy-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/export/apps/hadoop/site/etc/hadoop/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-java-1.1.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-format-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-all-4.1.17.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-ipc-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xmlenc-0.52.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jdo-api-3.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-client-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-runtime-3.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/pyrolite-4.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-catalyst_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-collections-3.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-api-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stream-2.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-format-2.3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-vector-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-web-proxy-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sketch_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-common-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hppc-0.7.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sql_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/univocity-parsers-2.5.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-math3-3.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compiler-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-1.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/java-xmlbuilder-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-annotations-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-3.9.9.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zookeeper-3.4.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-compiler-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/eigenbase-properties-1.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-yarn_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/JavaEWAH-0.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsr305-1.3.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libfb303-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.annotation-api-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-digester-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-jvm-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-framework-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/paranamer-2.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/janino-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-core-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-core-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsch-0.1.42.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-unsafe_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-codec-1.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jtransforms-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/lz4-java-1.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-core-3.2.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/core-1.1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/leveldbjni-all-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-dbcp-1.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang3-3.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill-java-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jodd-core-3.5.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-pool-1.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/minlog-1.3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/gson-2.2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/py4j-0.10.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-streaming_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/machinist_2.11-0.6.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-0.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-app-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-graphite-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-core-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib-local_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-memory-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-guava-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xercesImpl-2.9.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-tags_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javolution-5.5.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/joda-time-2.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-2.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-jobclient-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang-2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/compress-lzf-1.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-crypto-1.0.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-core-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-recipes-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guava-14.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-core_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-common_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-launcher_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr4-runtime-4.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-util-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jaxb-api-2.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-io-2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-encoding-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpcore-4.4.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-xc-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/protobuf-java-2.5.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-configuration-1.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compress-1.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-mapreduce-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ST4-4.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-core-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-shuffle-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-repl_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/opencsv-2.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-logging-1.1.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-cli-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-client-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-hdfs-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/log4j-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-column-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill_2.11-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stringtemplate-3.2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-common-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-shuffle_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-kvstore_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jta-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javassist-3.18.1-GA.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-httpclient-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jets3t-0.9.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-json-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bcprov-jdk15on-1.58.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/oro-2.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsp-api-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1506645932520_24630151","Timestamp":1521483660843,"User":"edlu"} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483693307,"Executor ID":"1","Executor Info":{"Host":"cluster-node4342.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stdout?start=-4096","stderr":"http://cluster-node4342.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000032/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483693370,"Executor ID":"3","Executor Info":{"Host":"cluster-node1177.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stdout?start=-4096","stderr":"http://cluster-node1177.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000038/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"cluster-node4342.company.com","Port":29144},"Maximum Memory":956615884,"Timestamp":1521483693405,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"cluster-node1177.company.com","Port":12614},"Maximum Memory":956615884,"Timestamp":1521483693483,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483701213,"Executor ID":"2","Executor Info":{"Host":"cluster-node6036.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stdout?start=-4096","stderr":"http://cluster-node6036.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000033/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"cluster-node6036.company.com","Port":17435},"Maximum Memory":956615884,"Timestamp":1521483701726,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483704950,"Executor ID":"4","Executor Info":{"Host":"cluster-node6137.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stdout?start=-4096","stderr":"http://cluster-node6137.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000040/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"cluster-node6137.company.com","Port":19036},"Maximum Memory":956615884,"Timestamp":1521483705369,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line44.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line44.$read$$iw$$iw$$iw$$iw.(:59)\n$line44.$read$$iw$$iw$$iw.(:61)\n$line44.$read$$iw$$iw.(:63)\n$line44.$read$$iw.(:65)\n$line44.$read.(:67)\n$line44.$read$.(:71)\n$line44.$read$.()\n$line44.$eval$.$print$lzycompute(:7)\n$line44.$eval$.$print(:6)\n$line44.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1521483706188} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1521483706193} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483707958,"JVM Used Memory":511901456,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":415523,"Offheap Storage Memory":0}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line48.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line48.$read$$iw$$iw$$iw$$iw.(:59)\n$line48.$read$$iw$$iw$$iw.(:61)\n$line48.$read$$iw$$iw.(:63)\n$line48.$read$$iw.(:65)\n$line48.$read.(:67)\n$line48.$read$.(:71)\n$line48.$read$.()\n$line48.$eval$.$print$lzycompute(:7)\n$line48.$eval$.$print(:6)\n$line48.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1521483708914} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1521483708914} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483712396,"JVM Used Memory":197797104,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at :40","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- AnalysisBarrier\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nappId: string, attemptId: string, name: string, mode: string, completed: string, duration: string, endTime: string, endTimeEpoch: string, lastUpdated: string, lastUpdatedEpoch: string, sparkUser: string, startTime: string, startTimeEpoch: string, appSparkVersion: string, endDate: string, azkaban.link.workflow.url: string, azkaban.link.execution.url: string, azkaban.link.job.url: string, user.name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@babef5e, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) LocalLimit 21\n +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@babef5e, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"LocalLimit","simpleString":"LocalLimit 21","children":[{"nodeName":"Project","simpleString":"Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"InMemoryTableScan","simpleString":"InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]","children":[],"metrics":[{"name":"number of output rows","accumulatorId":35,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":36,"metricType":"timing"}]}],"metrics":[]}],"metrics":[]}],"metrics":[]}],"metrics":[{"name":"duration total (min, med, max)","accumulatorId":34,"metricType":"timing"}]}],"metrics":[]},"time":1521483710146} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1521483710385,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]}],"Stage IDs":[0,1,2],"Properties":{"spark.sql.execution.id":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710427,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710514,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1521483710517,"Executor ID":"2","Host":"cluster-node6036.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"4","Host":"cluster-node6137.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483715514,"JVM Used Memory":83495872,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":36413,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483720047,"JVM Used Memory":121617408,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":36413,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483717971,"JVM Used Memory":603050152,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483722392,"JVM Used Memory":256639776,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483724389,"JVM Used Memory":431714816,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483725467,"JVM Used Memory":504156608,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":25165398,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483730046,"JVM Used Memory":292871096,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483727958,"JVM Used Memory":607069712,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1521483728324,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483728367,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"154334487","Value":"154334486","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"27714","Value":"27713","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":37809697,"Value":37809697,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":98699361,"Value":98699361,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":20002743,"Value":20002743,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":834,"Value":834,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":1856,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":13106272441,"Value":13106272441,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":15665,"Value":15665,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":741777109,"Value":741777109,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":2081,"Value":2081,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2081,"Executor Deserialize CPU Time":741777109,"Executor Run Time":15665,"Executor CPU Time":13106272441,"Result Size":1856,"JVM GC Time":834,"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":20002743,"Shuffle Write Time":98699361,"Shuffle Records Written":466636},"Input Metrics":{"Bytes Read":37809697,"Records Read":466636},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483732392,"JVM Used Memory":207788120,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":21875994,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483730182,"Executor ID":"7","Executor Info":{"Host":"cluster-node2449.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stdout?start=-4096","stderr":"http://cluster-node2449.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000064/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1521483730192,"Executor ID":"7","Host":"cluster-node2449.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"cluster-node2449.company.com","Port":26914},"Maximum Memory":956615884,"Timestamp":1521483730295,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483731267,"Executor ID":"5","Executor Info":{"Host":"cluster-node5882.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stdout?start=-4096","stderr":"http://cluster-node5882.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000043/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1521483731270,"Executor ID":"5","Host":"cluster-node5882.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"cluster-node5882.company.com","Port":24590},"Maximum Memory":956615884,"Timestamp":1521483731726,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483734387,"JVM Used Memory":585461008,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":1614952,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483735466,"JVM Used Memory":286822456,"Onheap Execution Memory":35651584,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483737958,"JVM Used Memory":619161088,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483739537,"Executor ID":"6","Executor Info":{"Host":"cluster-node6148.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stdout?start=-4096","stderr":"http://cluster-node6148.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000058/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1521483739540,"Executor ID":"6","Host":"cluster-node6148.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"cluster-node6148.company.com","Port":1233},"Maximum Memory":956615884,"Timestamp":1521483739859,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483742392,"JVM Used Memory":618421104,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":72624976,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1521483730192,"Executor ID":"7","Host":"cluster-node2449.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483741265,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"1953294","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"3575","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"76","Value":"75","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"7964","Value":"7963","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"196587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"3575","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"7495","Value":"7494","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":3575,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":36849246,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":71122623,"Value":71122623,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":3575,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":349006,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":942,"Value":942,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":2437,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5608539520,"Value":5608539520,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":9646,"Value":9646,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":775466204,"Value":775466204,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1177,"Value":1177,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1177,"Executor Deserialize CPU Time":775466204,"Executor Run Time":9646,"Executor CPU Time":5608539520,"Result Size":2437,"JVM GC Time":942,"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":349006,"Shuffle Write Time":71122623,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483744387,"JVM Used Memory":533322584,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":52049755,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1521483728324,"Executor ID":"1","Host":"cluster-node4342.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483741988,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"63","Value":"138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"13344","Value":"21307","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"12907","Value":"20401","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":43359041,"Value":114481664,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":102,"Value":1044,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4831,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":2885767088,"Value":8494306608,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":13538,"Value":23184,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":90660797,"Value":866127001,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":104,"Value":1281,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":104,"Executor Deserialize CPU Time":90660797,"Executor Run Time":13538,"Executor CPU Time":2885767088,"Result Size":2394,"JVM GC Time":102,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":43359041,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483745464,"JVM Used Memory":504693976,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483746631,"JVM Used Memory":365802552,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1521483745988,"Executor ID":"8","Executor Info":{"Host":"cluster-node6128.company.com","Total Cores":1,"Log Urls":{"stdout":"http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stdout?start=-4096","stderr":"http://cluster-node6128.company.com:8042/node/containerlogs/container_e02_1506645932520_24630151_01_000066/edlu/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"8","Host":"cluster-node6128.company.com","Port":1664},"Maximum Memory":956615884,"Timestamp":1521483746284,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483746534,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"206421303","Value":"360755789","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"65190","Value":"92903","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423609,"Value":88233306,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":123692332,"Value":222391693,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":26424033,"Value":46426776,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":878,"Value":1712,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":3712,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":19065776829,"Value":32172049270,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":35010,"Value":50675,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":732371255,"Value":1474148364,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":942,"Value":3023,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":942,"Executor Deserialize CPU Time":732371255,"Executor Run Time":35010,"Executor CPU Time":19065776829,"Result Size":1856,"JVM GC Time":878,"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":26424033,"Shuffle Write Time":123692332,"Shuffle Records Written":624246},"Input Metrics":{"Bytes Read":50423609,"Records Read":624246},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483750045,"JVM Used Memory":207536536,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":2461716,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483747958,"JVM Used Memory":641729640,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"5","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483751931,"JVM Used Memory":513138800,"Onheap Execution Memory":65536,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483752392,"JVM Used Memory":619676000,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483754387,"JVM Used Memory":695372944,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468552,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483755464,"JVM Used Memory":504694360,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483756630,"JVM Used Memory":365802904,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1521483710517,"Executor ID":"2","Host":"cluster-node6036.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483754578,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"73812","Value":"166715","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":214358240,"Value":436749933,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":3,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":4677,"Value":6389,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":24023445128,"Value":56195494398,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":42072,"Value":92747,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":952244001,"Value":2426392365,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":1796,"Value":4819,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1796,"Executor Deserialize CPU Time":952244001,"Executor Run Time":42072,"Executor CPU Time":24023445128,"Result Size":1856,"JVM GC Time":4677,"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":22950296,"Shuffle Write Time":214358240,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"6","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483758435,"JVM Used Memory":397749936,"Onheap Execution Memory":65536,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483760046,"JVM Used Memory":488354552,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":25910376,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483757958,"JVM Used Memory":662984224,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483762392,"JVM Used Memory":619676384,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"5","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483762915,"JVM Used Memory":353487552,"Onheap Execution Memory":41943040,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1521483731270,"Executor ID":"5","Host":"cluster-node5882.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483762215,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"5784372","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"10678","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"125829116","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"131","Value":"269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"22823","Value":"44130","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"586742","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"10679","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"19841","Value":"40242","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":10679,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":110532652,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":215191844,"Value":329673508,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":10678,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1053344,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":125829120,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":9,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":6111,"Value":7155,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":7268,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":8017651595,"Value":16511958203,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":27779,"Value":50963,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1058013355,"Value":1924140356,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":2357,"Value":3638,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2357,"Executor Deserialize CPU Time":1058013355,"Executor Run Time":27779,"Executor CPU Time":8017651595,"Result Size":2437,"JVM GC Time":6111,"Result Serialization Time":9,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355051,"Shuffle Write Time":215191844,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483765464,"JVM Used Memory":504694712,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483766630,"JVM Used Memory":365803256,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483767180,"JVM Used Memory":226589112,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"6","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483768433,"JVM Used Memory":501950360,"Onheap Execution Memory":35651584,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483770047,"JVM Used Memory":180202344,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":55709919,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483767958,"JVM Used Memory":665764240,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483772392,"JVM Used Memory":619676736,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483775464,"JVM Used Memory":504695064,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483776630,"JVM Used Memory":365803608,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1521483739540,"Executor ID":"6","Host":"cluster-node6148.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483774496,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"300","Value":"569","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"26291","Value":"70421","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"24092","Value":"64334","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":521474789,"Value":851148297,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":13,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":5484,"Value":12639,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9705,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":7239844597,"Value":23751802800,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":30603,"Value":81566,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1062050912,"Value":2986191268,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3790,"Value":7428,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3790,"Executor Deserialize CPU Time":1062050912,"Executor Run Time":30603,"Executor CPU Time":7239844597,"Result Size":2437,"JVM GC Time":5484,"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":242714,"Shuffle Write Time":521474789,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710514,"Completion Time":1521483774499,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":7428,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":23751802800,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":81566,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":851148297,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":12639,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2986191268,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"70421","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9705,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"569","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"64334","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483777175,"JVM Used Memory":226589144,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483780046,"JVM Used Memory":639104112,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":83580552,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483777958,"JVM Used Memory":669979432,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483782392,"JVM Used Memory":620936568,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483785464,"JVM Used Memory":505317000,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47961406,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"7","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483786630,"JVM Used Memory":365803960,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":562630,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"8","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483787175,"JVM Used Memory":226589176,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":0,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483790047,"JVM Used Memory":707255176,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69534656,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483787958,"JVM Used Memory":671982856,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":900126,"Offheap Storage Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483792392,"JVM Used Memory":620936920,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":62541048,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1521483710530,"Executor ID":"4","Host":"cluster-node6137.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1521483789895,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"131866","Value":"298581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":464179545,"Value":900929478,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":6979,"Value":13368,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":25321337014,"Value":81516831412,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":74896,"Value":167643,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1173542343,"Value":3599934708,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":4315,"Value":9134,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4315,"Executor Deserialize CPU Time":1173542343,"Executor Run Time":74896,"Executor CPU Time":25321337014,"Result Size":1856,"JVM GC Time":6979,"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":31362123,"Shuffle Write Time":464179545,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@321b0237, Location: InMemoryFileIndex[hdfs://clusternn01.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483710427,"Completion Time":1521483789896,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3599934708,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"298581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":81516831412,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":9134,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":900929478,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":167643,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":13368,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483789907,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1521483789924,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":1521483794387,"JVM Used Memory":317344968,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468552,"Offheap Storage Memory":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1521483789924,"Executor ID":"3","Host":"cluster-node1177.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1521483792118,"Failed":false,"Killed":false,"Accumulables":[{"ID":34,"Name":"duration total (min, med, max)","Update":"1","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":35,"Name":"number of output rows","Update":"6928","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Update":"1181","Value":"1180","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"number of output rows","Update":"10945","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"number of output rows","Update":"62","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Update":"33619967","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"sort time total (min, med, max)","Update":"6","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":22,"Name":"duration total (min, med, max)","Update":"896","Value":"895","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"peak memory total (min, med, max)","Update":"34078719","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Update":"53","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"duration total (min, med, max)","Update":"1109","Value":"1108","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Update":11007,"Value":11007,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":122827,"Value":122827,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":315848,"Value":315848,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":7,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Update":67698688,"Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":92,"Name":"internal.metrics.jvmGCTime","Update":146,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Update":4685,"Value":4685,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Update":970272606,"Value":970272606,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.executorRunTime","Update":1923,"Value":1923,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Update":162560302,"Value":162560302,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Update":230,"Value":230,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":230,"Executor Deserialize CPU Time":162560302,"Executor Run Time":1923,"Executor CPU Time":970272606,"Result Size":4685,"JVM GC Time":146,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":7,"Local Blocks Fetched":1,"Fetch Wait Time":1,"Remote Bytes Read":315848,"Remote Bytes Read To Disk":0,"Local Bytes Read":122827,"Total Records Read":11007},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":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":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1521483789907,"Completion Time":1521483792119,"Accumulables":[{"ID":92,"Name":"internal.metrics.jvmGCTime","Value":146,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Value":11007,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"number of output rows","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":89,"Name":"internal.metrics.executorRunTime","Value":1923,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":7,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"number of output rows","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":315848,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Value":4685,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"peak memory total (min, med, max)","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"duration total (min, med, max)","Value":"895","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Value":162560302,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"duration total (min, med, max)","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Value":"1180","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":19,"Name":"sort time total (min, med, max)","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Value":230,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Value":970272606,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"number of output rows","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Value":"52","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":122827,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"duration total (min, med, max)","Value":"1108","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1521483792127,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1521483792153} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node0033.grid.company.com","Port":60749},"Maximum Memory":1043437977,"Timestamp":1524182105107,"Maximum Onheap Memory":1043437977,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_31/jre","Java Version":"1.8.0_31 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.jars.ivySettings":"/export/apps/spark/commonconf/ivysettings.xml","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.driver.host":"node0033.grid.company.com","spark.dynamicAllocation.sustainedSchedulerBacklogTimeout":"5","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"57705","spark.shuffle.service.enabled":"true","spark.ui.acls.enable":"true","spark.reducer.maxSizeInFlight":"48m","spark.yarn.queue":"spark_default","spark.repl.class.uri":"spark://node0033.grid.company.com:57705/classes","spark.jars":"","spark.yarn.historyServer.address":"clustersh01.grid.company.com:18080","spark.memoryOverhead.multiplier.percent":"10","spark.repl.class.outputDir":"/grid/a/mapred/tmp/spark-21b68b4b-c1db-460e-a228-b87545d870f1/repl-58778a76-04c1-434d-bfb7-9a9b83afe718","spark.dynamicAllocation.cachedExecutorIdleTimeout":"1200","spark.yarn.access.namenodes":"hdfs://clusternn02.grid.company.com:9000","spark.app.name":"Spark shell","spark.dynamicAllocation.schedulerBacklogTimeout":"5","spark.yarn.security.credentials.hive.enabled":"false","spark.yarn.am.cores":"1","spark.memoryOverhead.min":"384","spark.scheduler.mode":"FIFO","spark.driver.memory":"2G","spark.executor.instances":"4","spark.isolated.classloader.additional.classes.prefix":"com_company_","spark.logConf":"true","spark.ui.showConsoleProgress":"true","spark.user.priority.jars":"*********(redacted)","spark.isolated.classloader":"true","spark.sql.sources.schemaStringLengthThreshold":"40000","spark.yarn.secondary.jars":"spark-avro_2.11-3.2.0.21.jar,grid-topology-1.0.jar","spark.reducer.maxBlocksInFlightPerAddress":"100","spark.dynamicAllocation.maxExecutors":"900","spark.yarn.appMasterEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.executor.id":"driver","spark.yarn.am.memory":"2G","spark.driver.cores":"1","spark.search.packages":"com.company.dali:dali-data-spark,com.company.spark-common:spark-common","spark.min.mem.vore.ratio":"5","spark.sql.sources.partitionOverwriteMode":"DYNAMIC","spark.submit.deployMode":"client","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.default.packages":"com.company.dali:dali-data-spark:8.+?classifier=all,com.company.spark-common:spark-common_2.10:0.+?","spark.isolated.classloader.default.jar":"*dali-data-spark*","spark.authenticate":"true","spark.eventLog.usexattr":"true","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.memory":"2G","spark.home":"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51","spark.reducer.maxReqsInFlight":"10","spark.eventLog.dir":"hdfs://clusternn02.grid.company.com:9000/system/spark-history","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.isolated.classes":"org.apache.hadoop.hive.ql.io.CombineHiveInputFormat$CombineHiveInputSplit","spark.eventLog.compress":"true","spark.executor.cores":"1","spark.version":"2.1.0","spark.driver.appUIAddress":"http://node0033.grid.company.com:8364","spark.repl.local.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"clusterwp01.grid.company.com","spark.min.memory-gb.size":"10","spark.dynamicAllocation.minExecutors":"1","spark.dynamicAllocation.initialExecutors":"3","spark.expressionencoder.org.apache.avro.specific.SpecificRecord":"com.databricks.spark.avro.AvroEncoder$","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://clusterwp01.grid.company.com:8080/proxy/application_1506645932520_24630151","spark.executorEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.dynamicAllocation.executorIdleTimeout":"150","spark.shell.auto.node.labeling":"true","spark.yarn.dist.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.app.id":"application_1506645932520_24630151","spark.ui.view.acls":"*"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_31/jre/lib/amd64","user.dir":"*********(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.31-b07","java.endorsed.dirs":"/usr/java/jdk1.8.0_31/jre/lib/endorsed","java.runtime.version":"1.8.0_31-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_31/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_31/jre/lib/resources.jar:/usr/java/jdk1.8.0_31/jre/lib/rt.jar:/usr/java/jdk1.8.0_31/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_31/jre/lib/jsse.jar:/usr/java/jdk1.8.0_31/jre/lib/jce.jar:/usr/java/jdk1.8.0_31/jre/lib/charsets.jar:/usr/java/jdk1.8.0_31/jre/lib/jfr.jar:/usr/java/jdk1.8.0_31/jre/classes","file.encoding":"UTF-8","user.timezone":"*********(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"2.6.32-504.16.2.el6.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*********(redacted)","sun.jnu.encoding":"UTF-8","user.language":"*********(redacted)","java.vendor.url":"*********(redacted)","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":"*********(redacted)","user.name":"*********(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --class org.apache.spark.repl.Main --name Spark shell --jars /export/home/edlu/spark-avro_2.11-3.2.0.21.jar,/export/apps/hadoop/site/lib/grid-topology-1.0.jar --num-executors 4 spark-shell","java.home":"/usr/java/jdk1.8.0_31/jre","java.version":"1.8.0_31","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-servlet-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/derby-10.12.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-reflect-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-graphx_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-util-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/base64-2.3.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-auth-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/validation-api-1.1.0.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-api-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/objenesis-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/conf/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpclient-4.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/kryo-shaded-3.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-library-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-net-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xz-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-annotations-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/activation-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arpack_combined_all-0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libthrift-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aircompressor-0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-jackson-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/asm-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-hive_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ivy-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/export/apps/hadoop/site/etc/hadoop/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-java-1.1.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-format-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-all-4.1.17.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-ipc-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xmlenc-0.52.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jdo-api-3.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-client-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-runtime-3.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/pyrolite-4.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-catalyst_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-collections-3.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-api-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stream-2.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-format-2.3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-vector-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-web-proxy-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sketch_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-common-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hppc-0.7.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sql_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/univocity-parsers-2.5.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-math3-3.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compiler-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-1.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/java-xmlbuilder-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-annotations-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-3.9.9.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zookeeper-3.4.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-compiler-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/eigenbase-properties-1.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-yarn_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/JavaEWAH-0.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsr305-1.3.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libfb303-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.annotation-api-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-digester-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-jvm-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-framework-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/paranamer-2.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/janino-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-core-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-core-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsch-0.1.42.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-unsafe_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-codec-1.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jtransforms-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/lz4-java-1.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-core-3.2.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/core-1.1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/leveldbjni-all-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-dbcp-1.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang3-3.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill-java-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jodd-core-3.5.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-pool-1.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/minlog-1.3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/gson-2.2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/py4j-0.10.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-streaming_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/machinist_2.11-0.6.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-0.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-app-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-graphite-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-core-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib-local_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-memory-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-guava-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xercesImpl-2.9.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-tags_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javolution-5.5.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/joda-time-2.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-2.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-jobclient-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang-2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/compress-lzf-1.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-crypto-1.0.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-core-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-recipes-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guava-14.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-core_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-common_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-launcher_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr4-runtime-4.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-util-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jaxb-api-2.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-io-2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-encoding-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpcore-4.4.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-xc-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/protobuf-java-2.5.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-configuration-1.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compress-1.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-mapreduce-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ST4-4.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-core-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-shuffle-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-repl_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/opencsv-2.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-logging-1.1.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-cli-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-client-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-hdfs-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/log4j-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-column-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill_2.11-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stringtemplate-3.2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-common-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-shuffle_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-kvstore_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jta-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javassist-3.18.1-GA.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-httpclient-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jets3t-0.9.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-json-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bcprov-jdk15on-1.58.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/oro-2.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsp-api-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1506645932520_24630151","Timestamp":1524182082734,"User":"edlu"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182111695,"Executor ID":"1","Executor Info":{"Host":"node1404.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stdout?start=-4096","stderr":"http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node1404.grid.company.com","Port":34043},"Maximum Memory":956615884,"Timestamp":1524182111795,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112088,"Executor ID":"3","Executor Info":{"Host":"node0998.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stdout?start=-4096","stderr":"http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"node0998.grid.company.com","Port":45265},"Maximum Memory":956615884,"Timestamp":1524182112208,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112278,"Executor ID":"4","Executor Info":{"Host":"node4243.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stdout?start=-4096","stderr":"http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"node4243.grid.company.com","Port":16084},"Maximum Memory":956615884,"Timestamp":1524182112408,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112471,"Executor ID":"2","Executor Info":{"Host":"node4045.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stdout?start=-4096","stderr":"http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"node4045.grid.company.com","Port":29262},"Maximum Memory":956615884,"Timestamp":1524182112578,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line44.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line44.$read$$iw$$iw$$iw$$iw.(:59)\n$line44.$read$$iw$$iw$$iw.(:61)\n$line44.$read$$iw$$iw.(:63)\n$line44.$read$$iw.(:65)\n$line44.$read.(:67)\n$line44.$read$.(:71)\n$line44.$read$.()\n$line44.$eval$.$print$lzycompute(:7)\n$line44.$eval$.$print(:6)\n$line44.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1524182125829} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1524182125832} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line48.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line48.$read$$iw$$iw$$iw$$iw.(:59)\n$line48.$read$$iw$$iw$$iw.(:61)\n$line48.$read$$iw$$iw.(:63)\n$line48.$read$$iw.(:65)\n$line48.$read.(:67)\n$line48.$read$.(:71)\n$line48.$read$.()\n$line48.$eval$.$print$lzycompute(:7)\n$line48.$eval$.$print(:6)\n$line48.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1524182128463} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1524182128463} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at :40","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- AnalysisBarrier\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nappId: string, attemptId: string, name: string, mode: string, completed: string, duration: string, endTime: string, endTimeEpoch: string, lastUpdated: string, lastUpdatedEpoch: string, sparkUser: string, startTime: string, startTimeEpoch: string, appSparkVersion: string, endDate: string, azkaban.link.workflow.url: string, azkaban.link.execution.url: string, azkaban.link.job.url: string, user.name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@485d3d1, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) LocalLimit 21\n +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@485d3d1, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"LocalLimit","simpleString":"LocalLimit 21","children":[{"nodeName":"Project","simpleString":"Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"InMemoryTableScan","simpleString":"InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]","children":[],"metrics":[{"name":"number of output rows","accumulatorId":35,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":36,"metricType":"timing"}]}],"metrics":[]}],"metrics":[]}],"metrics":[]}],"metrics":[{"name":"duration total (min, med, max)","accumulatorId":34,"metricType":"timing"}]}],"metrics":[]},"time":1524182129952} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1524182130194,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]}],"Stage IDs":[0,1,2],"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1524182130350,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1524182130350,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1524182130350,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182142286,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"154334487","Value":"154334486","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"466636","Value":"466636","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"19666","Value":"19665","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":37809697,"Value":37809697,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":91545212,"Value":91545212,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":466636,"Value":466636,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":20002743,"Value":20002743,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":407,"Value":407,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":1856,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":9020410971,"Value":9020410971,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":11146,"Value":11146,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":574344183,"Value":574344183,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":714,"Value":714,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":714,"Executor Deserialize CPU Time":574344183,"Executor Run Time":11146,"Executor CPU Time":9020410971,"Result Size":1856,"JVM GC Time":407,"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":20002743,"Shuffle Write Time":91545212,"Shuffle Records Written":466636},"Input Metrics":{"Bytes Read":37809697,"Records Read":466636},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1524182130350,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182143009,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"206421303","Value":"360755789","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"624246","Value":"1090882","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"20604","Value":"40269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423609,"Value":88233306,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":104125550,"Value":195670762,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":624246,"Value":1090882,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":26424033,"Value":46426776,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":374,"Value":781,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":3712,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11039226628,"Value":20059637599,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":11978,"Value":23124,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":526915936,"Value":1101260119,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":622,"Value":1336,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":622,"Executor Deserialize CPU Time":526915936,"Executor Run Time":11978,"Executor CPU Time":11039226628,"Result Size":1856,"JVM GC Time":374,"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":26424033,"Shuffle Write Time":104125550,"Shuffle Records Written":624246},"Input Metrics":{"Bytes Read":50423609,"Records Read":624246},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182143160,"Executor ID":"5","Executor Info":{"Host":"node2477.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stdout?start=-4096","stderr":"http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"node2477.grid.company.com","Port":20123},"Maximum Memory":956615884,"Timestamp":1524182143406,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","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":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144246,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"1920974","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"3562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"38","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1813","Value":"1812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"195602","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"3563","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1558","Value":"1557","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":3563,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":36845111,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":27318908,"Value":27318908,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":3562,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":349287,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":33,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1498974375,"Value":1498974375,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1922,"Value":1922,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":49547405,"Value":49547405,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":56,"Executor Deserialize CPU Time":49547405,"Executor Run Time":1922,"Executor CPU Time":1498974375,"Result Size":2394,"JVM GC Time":33,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":27318908,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144444,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"23365","Value":"63634","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":105575962,"Value":301246724,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":326,"Value":1107,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11931694025,"Value":31991331624,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13454,"Value":36578,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":531799977,"Value":1633060096,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":594,"Value":1930,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":594,"Executor Deserialize CPU Time":531799977,"Executor Run Time":13454,"Executor CPU Time":11931694025,"Result Size":1856,"JVM GC Time":326,"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":22950296,"Shuffle Write Time":105575962,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144840,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"24254","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":106963069,"Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":342,"Value":1449,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":12267596062,"Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13858,"Value":50436,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":519573839,"Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":573,"Value":2503,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":573,"Executor Deserialize CPU Time":519573839,"Executor Run Time":13858,"Executor CPU Time":12267596062,"Result Size":1856,"JVM GC Time":342,"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":31362123,"Shuffle Write Time":106963069,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":592412824,"JVM Used Nonheap Memory":202907152,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":905801,"Offheap Storage Memory":0,"Onheap Unified Memory":905801,"Offheap Unified Memory":0,"Direct Memory":355389,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":523121272,"JVM Used Nonheap Memory":88280720,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":52050147,"Offheap Storage Memory":0,"Onheap Unified Memory":52050147,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":214174608,"JVM Used Nonheap Memory":91548704,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47399168,"Offheap Storage Memory":0,"Onheap Unified Memory":47399168,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":518613056,"JVM Used Nonheap Memory":95657456,"Onheap Execution Memory":37748736,"Offheap Execution Memory":0,"Onheap Storage Memory":63104457,"Offheap Storage Memory":0,"Onheap Unified Memory":100853193,"Offheap Unified Memory":0,"Direct Memory":126261,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":726805712,"JVM Used Nonheap Memory":90709624,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69535048,"Offheap Storage Memory":0,"Onheap Unified Memory":69535048,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Completion Time":1524182144852,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":2503,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":50436,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":1449,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145327,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"49","Value":"86","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"2002","Value":"3814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1755","Value":"3312","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":32035583,"Value":59354491,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4788,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1785119941,"Value":3284094316,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":2182,"Value":4104,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":71500541,"Value":121047946,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":136,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":136,"Executor Deserialize CPU Time":71500541,"Executor Run Time":2182,"Executor CPU Time":1785119941,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349006,"Shuffle Write Time":32035583,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145971,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"5212268","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"9572","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"121634812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"9","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1703","Value":"5517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"525948","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"9573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1609","Value":"4921","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":9573,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":97944567,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":20055909,"Value":79410400,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":9572,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":941007,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":121634816,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":7182,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":896878991,"Value":4180973307,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1722,"Value":5826,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2787355,"Value":123835301,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":195,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2787355,"Executor Run Time":1722,"Executor CPU Time":896878991,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":242714,"Shuffle Write Time":20055909,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182147549,"Executor ID":"6","Executor Info":{"Host":"node6644.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stdout?start=-4096","stderr":"http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"node6644.grid.company.com","Port":8445},"Maximum Memory":956615884,"Timestamp":1524182147706,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1524182149826,"Executor ID":"7","Executor Info":{"Host":"node6340.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096","stderr":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"node6340.grid.company.com","Port":5933},"Maximum Memory":956615884,"Timestamp":1524182149983,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182152418,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"48","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"6093","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"5951","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":49790497,"Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":920,"Value":1015,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9619,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5299274511,"Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":7847,"Value":13673,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687811857,"Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1037,"Value":1232,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1037,"Executor Deserialize CPU Time":687811857,"Executor Run Time":7847,"Executor CPU Time":5299274511,"Result Size":2437,"JVM GC Time":920,"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":355051,"Shuffle Write Time":49790497,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":629553808,"JVM Used Nonheap Memory":205304696,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":905801,"Offheap Storage Memory":0,"Onheap Unified Memory":905801,"Offheap Unified Memory":0,"Direct Memory":397602,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":595946552,"JVM Used Nonheap Memory":91208368,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468944,"Offheap Storage Memory":0,"Onheap Unified Memory":58468944,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":755008624,"JVM Used Nonheap Memory":100519936,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47962185,"Offheap Storage Memory":0,"Onheap Unified Memory":47962185,"Offheap Unified Memory":0,"Direct Memory":98230,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":518613056,"JVM Used Nonheap Memory":95657456,"Onheap Execution Memory":37748736,"Offheap Execution Memory":0,"Onheap Storage Memory":63104457,"Offheap Storage Memory":0,"Onheap Unified Memory":100853193,"Offheap Unified Memory":0,"Direct Memory":126261,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":726805712,"JVM Used Nonheap Memory":90709624,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69535048,"Offheap Storage Memory":0,"Onheap Unified Memory":69535048,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Completion Time":1524182152419,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":1232,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":13673,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":1015,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9619,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1524182152447,"Executor ID":"4","Host":"node4243.grid.company.com","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":8,"Index":0,"Attempt":0,"Launch Time":1524182152447,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1524182153103,"Failed":false,"Killed":false,"Accumulables":[{"ID":34,"Name":"duration total (min, med, max)","Update":"1","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":35,"Name":"number of output rows","Update":"6928","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Update":"452","Value":"451","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":11,"Name":"number of output rows","Update":"10945","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":18,"Name":"number of output rows","Update":"62","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Update":"33619967","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":22,"Name":"duration total (min, med, max)","Update":"323","Value":"322","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":13,"Name":"peak memory total (min, med, max)","Update":"34078719","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Update":"10","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":15,"Name":"duration total (min, med, max)","Update":"367","Value":"366","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Update":11007,"Value":11007,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":124513,"Value":124513,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":314162,"Value":314162,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":6,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Update":67698688,"Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":91,"Name":"internal.metrics.resultSize","Update":4642,"Value":4642,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Update":517655714,"Value":517655714,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.executorRunTime","Update":589,"Value":589,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Update":45797784,"Value":45797784,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Update":50,"Value":50,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":50,"Executor Deserialize CPU Time":45797784,"Executor Run Time":589,"Executor CPU Time":517655714,"Result Size":4642,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":6,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":314162,"Remote Bytes Read To Disk":0,"Local Bytes Read":124513,"Total Records Read":11007},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":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":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Completion Time":1524182153104,"Accumulables":[{"ID":101,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.shuffle.read.recordsRead","Value":11007,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"number of output rows","Value":"6928","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":89,"Name":"internal.metrics.executorRunTime","Value":589,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":6,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"number of output rows","Value":"10945","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"peak memory total (min, med, max)","Value":"33619966","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":91,"Name":"internal.metrics.resultSize","Value":4642,"Internal":true,"Count Failed Values":true},{"ID":100,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":314162,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"peak memory total (min, med, max)","Value":"34078718","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":22,"Name":"duration total (min, med, max)","Value":"322","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":103,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorDeserializeCpuTime","Value":45797784,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"duration total (min, med, max)","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":10,"Name":"duration total (min, med, max)","Value":"451","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":87,"Name":"internal.metrics.executorDeserializeTime","Value":50,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.peakExecutionMemory","Value":67698688,"Internal":true,"Count Failed Values":true},{"ID":90,"Name":"internal.metrics.executorCpuTime","Value":517655714,"Internal":true,"Count Failed Values":true},{"ID":99,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":2,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"number of output rows","Value":"62","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":12,"Name":"sort time total (min, med, max)","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":102,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":124513,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"duration total (min, med, max)","Value":"366","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1524182153112,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1524182153139} {"Event":"SparkListenerUnpersistRDD","RDD ID":2} {"Event":"SparkListenerUnpersistRDD","RDD ID":20} -{"Event":"SparkListenerApplicationEnd","Timestamp":1521483792501} +{"Event":"SparkListenerApplicationEnd","Timestamp":1524182189134} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 4f8081f2e942..b2d4505eaf87 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -213,7 +213,8 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) - val executorUpdates = new ExecutorMetrics(1234567L, 123456L, 12345L, 1234L, 123L, 12L) + val executorUpdates = new ExecutorMetrics(1234567L, 123456L, 543L, 12345L, 1234L, 123L, 12L, + 432L, 321L, 654L, 765L) val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId, executorUpdates)) if (executorShouldReregister) { 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 59645b2cf181..6b906a88183c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -48,6 +48,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} */ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter with Logging { + import EventLoggingListenerSuite._ private val fileSystem = Utils.getHadoopFileSystem("/", @@ -273,54 +274,95 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus(conf) - // list of events and if they should be logged + // expected ExecutorMetricsUpdate, for the given stage id and executor id + val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] = + Map( + ((0, "1"), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))), + ((0, "2"), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))), + ((1, "1"), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))), + ((1, "2"), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + + // Events to post. val events = Array( - (SparkListenerApplicationStart("executionMetrics", None, - 1L, "update", None), true), - (createExecutorAddedEvent(1), true), - (createExecutorAddedEvent(2), true), - (createStageSubmittedEvent(0), true), - (createExecutorMetricsUpdateEvent(1, 10L, 5000L, 50L, 0L, 0L, 0L), true), // new stage - (createExecutorMetricsUpdateEvent(2, 10L, 3500L, 20L, 0L, 0L, 0L), true), // new stage - (createExecutorMetricsUpdateEvent(1, 15L, 4000L, 50L, 0L, 0L, 0L), false), - (createExecutorMetricsUpdateEvent(2, 15L, 3500L, 10L, 0L, 20L, 0L), true), // onheap storage - (createExecutorMetricsUpdateEvent(1, 20L, 6000L, 50L, 0L, 30L, 0L), true), // JVM used - (createExecutorMetricsUpdateEvent(2, 20L, 3500L, 15L, 0L, 20L, 0L), true), // onheap unified - (createStageSubmittedEvent(1), true), - (createExecutorMetricsUpdateEvent(1, 25L, 3000L, 15L, 0L, 0L, 0L), true), // new stage - (createExecutorMetricsUpdateEvent(2, 25L, 6000L, 50L, 0L, 0L, 0L), true), // new stage - (createStageCompletedEvent(0), true), - (createExecutorMetricsUpdateEvent(1, 30L, 3000L, 20L, 0L, 0L, 0L), true), // onheap execution - (createExecutorMetricsUpdateEvent(2, 30L, 5500L, 20L, 0L, 0L, 0L), false), - (createExecutorMetricsUpdateEvent(1, 35L, 3000L, 5L, 25L, 0L, 0L), true), // offheap execution - (createExecutorMetricsUpdateEvent(2, 35L, 5500L, 25L, 0L, 0L, 30L), true), // offheap storage - (createExecutorMetricsUpdateEvent(1, 40L, 3000L, 8L, 20L, 0L, 0L), false), - (createExecutorMetricsUpdateEvent(2, 40L, 5500L, 25L, 0L, 0L, 30L), false), - (createStageCompletedEvent(1), true), - (SparkListenerApplicationEnd(1000L), true)) + SparkListenerApplicationStart("executionMetrics", None, + 1L, "update", None), + createExecutorAddedEvent(1), + createExecutorAddedEvent(2), + createStageSubmittedEvent(0), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)), + createStageSubmittedEvent(1), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)), + createStageCompletedEvent(0), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)), + createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)), + createExecutorRemovedEvent(1), + createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)), + createStageCompletedEvent(1), + SparkListenerApplicationEnd(1000L)) // play the events for the event logger eventLogger.start() listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem])) listenerBus.addToEventLogQueue(eventLogger) - for ((event, included) <- events) { - listenerBus.post(event) - } + events.foreach(event => listenerBus.post(event)) listenerBus.stop() eventLogger.stop() - // Verify the log file contains the expected events + // Verify the log file contains the expected events. + // Posted events should be logged, except for ExecutorMetricsUpdate events -- these + // are consolidated, and the peak values for each stage are logged at stage end. val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) try { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) - assert(lines.size === 19) + assert(lines.size === 14) assert(lines(0).contains("SparkListenerLogStart")) assert(lines(1).contains("SparkListenerApplicationStart")) assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) var i = 1 - for ((event, included) <- events) { - if (included) { + events.foreach {event => + event match { + case metricsUpdate: SparkListenerExecutorMetricsUpdate => + case stageCompleted: SparkListenerStageCompleted => + for (j <- 1 to 2) { + checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId, + expectedMetricsEvents) + i += 1 + } + checkEvent(lines(i), event) + i += 1 + case _ => checkEvent(lines(i), event) i += 1 } @@ -331,55 +373,58 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } /** Create a stage submitted event for the specified stage Id. */ - private def createStageSubmittedEvent(stageId: Int) = + private def createStageSubmittedEvent(stageId: Int) = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "details")) + } /** Create a stage completed event for the specified stage Id. */ - private def createStageCompletedEvent(stageId: Int) = + private def createStageCompletedEvent(stageId: Int) = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "details")) + } /** Create an executor added event for the specified executor Id. */ - private def createExecutorAddedEvent(executorId: Int) = + private def createExecutorAddedEvent(executorId: Int) = { SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty)) + } + + /** Create an executor added event for the specified executor Id. */ + private def createExecutorRemovedEvent(executorId: Int) = { + SparkListenerExecutorRemoved(0L, executorId.toString, "test") + } /** Create an executor metrics update event, with the specified executor metrics values. */ private def createExecutorMetricsUpdateEvent( - executorId: Int, time: Long, - jvmUsedMemory: Long, - onHeapExecutionMemory: Long, - offHeapExecutionMemory: Long, - onHeapStorageMemory: Long, - offHeapStorageMemory: Long): SparkListenerExecutorMetricsUpdate = { + executorId: Int, + executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { val taskMetrics = TaskMetrics.empty taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) - val executorUpdates = new ExecutorMetrics(time, jvmUsedMemory, onHeapExecutionMemory, - offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory) - SparkListenerExecutorMetricsUpdate( executorId.toString, accum, Some(executorUpdates)) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) } /** Check that the two ExecutorMetrics match */ private def checkExecutorMetrics( - executorMetrics1: Option[ExecutorMetrics], - executorMetrics2: Option[ExecutorMetrics]) = { - executorMetrics1 match { - case Some(e1) => - executorMetrics2 match { - case Some(e2) => - assert(e1.timestamp === e2.timestamp) - assert(e1.jvmUsedMemory === e2.jvmUsedMemory) - assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory) - assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory) - assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory) - assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory) - } - case None => + executorMetrics1: Option[ExecutorMetrics], + executorMetrics2: Option[ExecutorMetrics]) = { + (executorMetrics1, executorMetrics2) match { + case (Some(e1), Some(e2)) => + assert(e1.timestamp === e2.timestamp) + assert(e1.jvmUsedHeapMemory === e2.jvmUsedHeapMemory) + assert(e1.jvmUsedNonHeapMemory === e2.jvmUsedNonHeapMemory) + assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory) + assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory) + assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory) + assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory) + assert(e1.onHeapUnifiedMemory === e2.onHeapUnifiedMemory) + assert(e1.offHeapUnifiedMemory === e2.offHeapUnifiedMemory) + assert(e1.directMemory === e2.directMemory) + assert(e1.mappedMemory === e2.mappedMemory) + case (None, None) => + case _ => assert(false) - case None => - assert(executorMetrics2.isEmpty) } } @@ -417,6 +462,32 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } + /** + * Check that the Spark history log line is an ExecutorMetricsUpdate, and matches the expected + * value for the stage and executor. + * + * @param line the Spark history log line + * @param stageId the stage ID the ExecutorMetricsUpdate is associated with + * @param expectedEvents map of expected ExecutorMetricsUpdate events, for (stageId, executorId) + */ + private def checkExecutorMetricsUpdate( + line: String, + stageId: Int, + expectedEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate]): Unit = { + JsonProtocol.sparkEventFromJson(parse(line)) match { + case executorMetrics: SparkListenerExecutorMetricsUpdate => + expectedEvents.get((stageId, executorMetrics.execId)) match { + case Some(expectedMetrics) => + assert(executorMetrics.accumUpdates.isEmpty) + checkExecutorMetrics(executorMetrics.executorUpdates, expectedMetrics.executorUpdates) + case None => + assert(false) + } + case _ => + assertTypeError("expecting SparkListenerExecutorMetricsUpdate") + } + } + private def readLines(in: InputStream): Seq[String] = { Source.fromInputStream(in).getLines().toSeq } 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 1cd71955ad4d..136f20f8db04 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -22,18 +22,19 @@ import java.lang.{Integer => JInteger, Long => JLong} import java.util.{Arrays, Date, Properties} import scala.collection.JavaConverters._ +import scala.collection.immutable.Map import scala.reflect.{classTag, ClassTag} import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.api.v1 +import org.apache.spark.status.api.v1.PeakMemoryMetrics import org.apache.spark.storage._ import org.apache.spark.util.Utils -import org.apache.spark.util.kvstore._ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { @@ -1208,6 +1209,75 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } } + test("executor metrics updates") { + val listener = new AppStatusListener(store, conf, true) + + val driver = BlockManagerId(SparkContext.DRIVER_IDENTIFIER, "localhost", 42) + + listener.onExecutorAdded(createExecutorAddedEvent(1)) + listener.onExecutorAdded(createExecutorAddedEvent(2)) + listener.onStageSubmitted(createStageSubmittedEvent(0)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + listener.onStageSubmitted(createStageSubmittedEvent(1)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(25L, 7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + listener.onStageCompleted(createStageCompletedEvent(0)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + listener.onExecutorRemoved(createExecutorRemovedEvent(1)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + listener.onStageCompleted(createStageCompletedEvent(1)) + + // expected peak values for each executor + val expectedValues = Map( + "1" -> new PeakMemoryMetrics(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), + "2" -> new PeakMemoryMetrics(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) + + // check that the stored peak values match the expected values + for ((id, metrics) <- expectedValues) { + check[ExecutorSummaryWrapper](id) { exec => + assert(exec.info.id === id) + exec.info.peakMemoryMetrics match { + case Some(actual) => + assert(actual.jvmUsedHeapMemory == metrics.jvmUsedHeapMemory) + assert(actual.jvmUsedNonHeapMemory == metrics.jvmUsedNonHeapMemory) + assert(actual.onHeapExecutionMemory == metrics.onHeapExecutionMemory) + assert(actual.offHeapExecutionMemory == metrics.offHeapExecutionMemory) + assert(actual.onHeapStorageMemory == metrics.onHeapStorageMemory) + assert(actual.offHeapStorageMemory == metrics.offHeapStorageMemory) + assert(actual.onHeapUnifiedMemory == metrics.onHeapUnifiedMemory) + assert(actual.offHeapUnifiedMemory == metrics.offHeapUnifiedMemory) + assert(actual.directMemory == metrics.directMemory) + assert(actual.mappedMemory == metrics.mappedMemory) + case _ => + assert(false) + } + } + } + } + private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptNumber) private def check[T: ClassTag](key: Any)(fn: T => Unit): Unit = { @@ -1245,4 +1315,36 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } + /** Create a stage submitted event for the specified stage Id. */ + private def createStageSubmittedEvent(stageId: Int) = { + SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + } + + /** Create a stage completed event for the specified stage Id. */ + private def createStageCompletedEvent(stageId: Int) = { + SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, + Seq.empty, Seq.empty, "details")) + } + + /** 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)) + } + + /** Create an executor added event for the specified executor Id. */ + private def createExecutorRemovedEvent(executorId: Int) = { + SparkListenerExecutorRemoved(0L, executorId.toString, "test") + } + + /** Create an executor metrics update event, with the specified executor metrics values. */ + private def createExecutorMetricsUpdateEvent( + executorId: Int, + executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { + val taskMetrics = TaskMetrics.empty + taskMetrics.incDiskBytesSpilled(111) + taskMetrics.incMemoryBytesSpilled(222) + val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) + } } 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 e0cc3c46017c..afe11afe3e5c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -94,7 +94,8 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - val executorUpdates = Some(new ExecutorMetrics(1234567L, 123456L, 12345L, 1234L, 123L, 12L)) + val executorUpdates = Some(new ExecutorMetrics(1234567L, 543L, 123456L, 12345L, 1234L, 123L, + 12L, 432L, 321L, 654L, 765L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) } val blockUpdated = @@ -682,21 +683,18 @@ private[spark] object JsonProtocolSuite extends Assertions { } private def assertEquals(metrics1: Option[ExecutorMetrics], metrics2: Option[ExecutorMetrics]) { - metrics1 match { - case Some(m1) => - metrics2 match { - case Some(m2) => - assert(m1.timestamp === m2.timestamp) - assert(m1.jvmUsedMemory === m2.jvmUsedMemory) - assert(m1.onHeapExecutionMemory === m2.onHeapExecutionMemory) - assert(m1.offHeapExecutionMemory === m2.offHeapExecutionMemory) - assert(m1.onHeapStorageMemory === m2.onHeapStorageMemory) - assert(m1.offHeapStorageMemory === m2.offHeapStorageMemory) - case None => - assert(false) - } - case None => - assert(metrics2.isEmpty) + (metrics1, metrics2) match { + case (Some(m1), Some(m2)) => + assert(m1.timestamp === m2.timestamp) + assert(m1.jvmUsedHeapMemory === m2.jvmUsedHeapMemory) + assert(m1.jvmUsedNonHeapMemory === m2.jvmUsedNonHeapMemory) + assert(m1.onHeapExecutionMemory === m2.onHeapExecutionMemory) + assert(m1.offHeapExecutionMemory === m2.offHeapExecutionMemory) + assert(m1.onHeapStorageMemory === m2.onHeapStorageMemory) + assert(m1.offHeapStorageMemory === m2.offHeapStorageMemory) + case (None, None) => + case _ => + assert(false) } } @@ -867,9 +865,10 @@ private[spark] object JsonProtocolSuite extends Assertions { internal, countFailedValues, metadata) /** Creates an SparkListenerExecutorMetricsUpdate event */ - private def makeExecutorMetricsUpdate(execId: String, - includeTaskMetrics: Boolean, - includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { + private def makeExecutorMetricsUpdate( + execId: String, + includeTaskMetrics: Boolean, + includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { val taskMetrics = if (includeTaskMetrics) { Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), @@ -879,7 +878,7 @@ private[spark] object JsonProtocolSuite extends Assertions { } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(1234567L, 123456L, 0L, 0L, 0L, 0L)) + Some(new ExecutorMetrics(1234567L, 123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L)) } else { None } @@ -2076,11 +2075,16 @@ private[spark] object JsonProtocolSuite extends Assertions { | ], | "Executor Metrics Updated" : { | "Timestamp" : 1234567, - | "JVM Used Memory" : 123456, + | "JVM Used Heap Memory" : 543, + | "JVM Used Nonheap Memory" : 123456, | "Onheap Execution Memory" : 12345, | "Offheap Execution Memory" : 1234, | "Onheap Storage Memory" : 123, - | "Offheap Storage Memory" : 12 + | "Offheap Storage Memory" : 12, + | "Onheap Unified Memory" : 432, + | "Offheap Unified Memory" : 321, + | "Direct Memory" : 654, + | "Mapped Memory" : 765 | } | |} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ebca8c85deec..362ccb345254 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -79,6 +79,7 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.Node"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.this"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.this"), +<<<<<<< HEAD // [SPARK-7132][ML] Add fit with validation set to spark.ml GBT ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), @@ -90,6 +91,8 @@ object MimaExcludes { ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), +======= +>>>>>>> Address code review comments, change event logging to stage end. // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"), From ca85c8219f46e3265b8191e82a4017c2cb97fc49 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Mon, 14 May 2018 17:24:22 -0700 Subject: [PATCH 12/29] Add configuration parameter spark.eventLog.logExecutorMetricsUpdates.enabled to enable/disable executor metrics update logging. Code review comments. --- .../scala/org/apache/spark/Heartbeater.scala | 5 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/internal/config/package.scala | 5 ++ .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scheduler/EventLoggingListener.scala | 66 +++++++++++-------- 5 files changed, 50 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index b36285ed42e6..2c89cc14189d 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -26,11 +26,12 @@ import org.apache.spark.util.{ThreadUtils, Utils} * intervals of intervalMs. * * @param reportHeartbeat the heartbeat reporting function to call. + * @param name the thread name for the heartbeater. * @param intervalMs the interval between heartbeats. */ -private[spark] class Heartbeater(reportHeartbeat: () => Unit, intervalMs: Long) { +private[spark] class Heartbeater(reportHeartbeat: () => Unit, name: String, intervalMs: Long) { // Executor for the heartbeat task - private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater") + private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor(name) /** Schedules a task to report a heartbeat. */ private[spark] def start(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 7cabeee14d43..7012a20c3f17 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -155,7 +155,7 @@ private[spark] class Executor( private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] // Executor for the heartbeat task. - private val heartbeater = new Heartbeater(reportHeartBeat, + private val heartbeater = new Heartbeater(reportHeartBeat, "executor-heartbeater", conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) // must be initialized before running startDriverHeartbeat() diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index a54b091a64d5..8c0bdc5b1c09 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -69,6 +69,11 @@ package object config { .bytesConf(ByteUnit.KiB) .createWithDefaultString("100k") + private[spark] val EVENT_LOG_EXECUTOR_METRICS_UPDATES = + ConfigBuilder("spark.eventLog.logExecutorMetricsUpdates.enabled") + .booleanConf + .createWithDefault(true) + private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 762c7d0bd719..b7dc716ec1c0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -211,7 +211,7 @@ class DAGScheduler( taskScheduler.setDAGScheduler(this) /** driver heartbeat for collecting metrics */ - private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, + private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, "driver-heartbeater", sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) /** BufferPoolMXBean for direct memory */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 5fae14bb1708..a795186f20b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -52,6 +52,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} * spark.eventLog.overwrite - Whether to overwrite any existing files. * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams + * spark.eventLog.logExecutorMetricsUpdates.enabled - Whether to log executor metrics updates */ private[spark] class EventLoggingListener( appId: String, @@ -70,6 +71,7 @@ private[spark] class EventLoggingListener( private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) + private val shouldLogExecutorMetricsUpdates = sparkConf.get(EVENT_LOG_EXECUTOR_METRICS_UPDATES) private val testing = sparkConf.get(EVENT_LOG_TESTING) private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) @@ -82,7 +84,7 @@ private[spark] class EventLoggingListener( private val compressionCodecName = compressionCodec.map { c => CompressionCodec.getShortName(c.getClass.getName) } - +logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecutorMetricsUpdates) // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None @@ -162,9 +164,11 @@ private[spark] class EventLoggingListener( // Events that do not trigger a flush override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { logEvent(event) - // clear the peak metrics when a new stage starts - liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), - new mutable.HashMap[String, PeakExecutorMetrics]()) + if (shouldLogExecutorMetricsUpdates) { + // record the peak metrics for the new stage + liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), + new mutable.HashMap[String, PeakExecutorMetrics]()) + } } override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) @@ -179,22 +183,30 @@ private[spark] class EventLoggingListener( // Events that trigger a flush override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { - // log the peak executor metrics for the stage, for each executor - val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() - val executorMap = liveStageExecutorMetrics.remove( - (event.stageInfo.stageId, event.stageInfo.attemptNumber())) - executorMap.foreach { - executorEntry => { - for ((executorId, peakExecutorMetrics) <- executorEntry) { - val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory, - peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory, - peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory, - peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory, - peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory, - peakExecutorMetrics.mappedMemory) - val executorUpdate = new SparkListenerExecutorMetricsUpdate( - executorId, accumUpdates, Some(executorMetrics)) - logEvent(executorUpdate) + if (shouldLogExecutorMetricsUpdates) { + // clear out any previous attempts, that did not have a stage completed event + val prevAttemptId = event.stageInfo.attemptNumber() - 1 + for (attemptId <- 0 to prevAttemptId) { + liveStageExecutorMetrics.remove((event.stageInfo.stageId, attemptId)) + } + + // log the peak executor metrics for the stage, for each executor + val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() + val executorMap = liveStageExecutorMetrics.remove( + (event.stageInfo.stageId, event.stageInfo.attemptNumber())) + executorMap.foreach { + executorEntry => { + for ((executorId, peakExecutorMetrics) <- executorEntry) { + val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory, + peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory, + peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory, + peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory, + peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory, + peakExecutorMetrics.mappedMemory) + val executorUpdate = new SparkListenerExecutorMetricsUpdate( + executorId, accumUpdates, Some(executorMetrics)) + logEvent(executorUpdate) + } } } } @@ -266,12 +278,14 @@ private[spark] class EventLoggingListener( } override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - // For the active stages, record any new peak values for the memory metrics for the executor - event.executorUpdates.foreach { executorUpdates => - liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => - val peakMetrics = peakExecutorMetrics.getOrElseUpdate( - event.execId, new PeakExecutorMetrics()) - peakMetrics.compareAndUpdate(executorUpdates) + if (shouldLogExecutorMetricsUpdates) { + // For the active stages, record any new peak values for the memory metrics for the executor + event.executorUpdates.foreach { executorUpdates => + liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => + val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + event.execId, new PeakExecutorMetrics()) + peakMetrics.compareAndUpdate(executorUpdates) + } } } } From 8b74ba8fff21b499e7cc9d93f9864831aa29773e Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 23 May 2018 14:37:26 -0500 Subject: [PATCH 13/29] wip on enum based metrics --- .../apache/spark/scheduler/MemoryTypes.java | 67 +++++++++++++ .../spark/scheduler/PeakExecutorMetrics.scala | 93 ++++--------------- .../org/apache/spark/status/api/v1/api.scala | 17 +--- 3 files changed, 90 insertions(+), 87 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java diff --git a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java new file mode 100644 index 000000000000..876233916f0b --- /dev/null +++ b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java @@ -0,0 +1,67 @@ +/* + * 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.scheduler; + +import org.apache.spark.executor.ExecutorMetrics; + +public enum MemoryTypes { + JvmUsedMemory{ + @Override + long get(ExecutorMetrics em) { + return em.jvmUsedMemory(); + } + }, + OnHeapExecutionMemory { + @Override + long get(ExecutorMetrics em) { + return em.onHeapExecutionMemory(); + } + }, + OffHeapExecutionMemory { + @Override + long get(ExecutorMetrics em) { + return em.offHeapExecutionMemory(); + } + }, + OnHeapStorageMemory { + @Override + long get(ExecutorMetrics em) { + return em.onHeapStorageMemory(); + } + }, + OffHeapStorageMemory { + @Override + long get(ExecutorMetrics em) { + return em.offHeapStorageMemory(); + } + }, + OnHeapUnifiedMemory { + @Override + long get(ExecutorMetrics em) { + return em.onHeapExecutionMemory() + em.onHeapStorageMemory(); + } + }, + OffHeapUnifiedMemory { + @Override + long get(ExecutorMetrics em) { + return em.offHeapExecutionMemory() + em.offHeapStorageMemory(); + } + }; + + abstract long get(ExecutorMetrics em); +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index d554c938f5e1..a91aabde9306 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -25,36 +25,8 @@ import org.apache.spark.status.api.v1.PeakMemoryMetrics * values have been recorded yet. */ private[spark] class PeakExecutorMetrics { - private var _jvmUsedHeapMemory = -1L; - private var _jvmUsedNonHeapMemory = 0L; - private var _onHeapExecutionMemory = 0L - private var _offHeapExecutionMemory = 0L - private var _onHeapStorageMemory = 0L - private var _offHeapStorageMemory = 0L - private var _onHeapUnifiedMemory = 0L - private var _offHeapUnifiedMemory = 0L - private var _directMemory = 0L - private var _mappedMemory = 0L - - def jvmUsedHeapMemory: Long = _jvmUsedHeapMemory - - def jvmUsedNonHeapMemory: Long = _jvmUsedNonHeapMemory - - def onHeapExecutionMemory: Long = _onHeapExecutionMemory - - def offHeapExecutionMemory: Long = _offHeapExecutionMemory - - def onHeapStorageMemory: Long = _onHeapStorageMemory - - def offHeapStorageMemory: Long = _offHeapStorageMemory - - def onHeapUnifiedMemory: Long = _onHeapUnifiedMemory - - def offHeapUnifiedMemory: Long = _offHeapUnifiedMemory - - def directMemory: Long = _directMemory - - def mappedMemory: Long = _mappedMemory + val metrics = new Array[Long](MemoryTypes.values().length) + metrics(0) = -1 /** * Compare the specified memory values with the saved peak executor memory @@ -66,47 +38,13 @@ private[spark] class PeakExecutorMetrics { def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { var updated: Boolean = false - if (executorMetrics.jvmUsedHeapMemory > _jvmUsedHeapMemory) { - _jvmUsedHeapMemory = executorMetrics.jvmUsedHeapMemory - updated = true - } - if (executorMetrics.jvmUsedNonHeapMemory > _jvmUsedNonHeapMemory) { - _jvmUsedNonHeapMemory = executorMetrics.jvmUsedNonHeapMemory - updated = true + (0 until MemoryTypes.values().length).foreach { metricIdx => + val metricVal = MemoryTypes.values()(metricIdx).get(executorMetrics) + if (metricVal > metrics(metricIdx)) { + updated = true + metrics(metricIdx) = metricVal + } } - if (executorMetrics.onHeapExecutionMemory > _onHeapExecutionMemory) { - _onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory - updated = true - } - if (executorMetrics.offHeapExecutionMemory > _offHeapExecutionMemory) { - _offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory - updated = true - } - if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) { - _onHeapStorageMemory = executorMetrics.onHeapStorageMemory - updated = true - } - if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) { - _offHeapStorageMemory = executorMetrics.offHeapStorageMemory - updated = true - } - if (executorMetrics.onHeapUnifiedMemory > _onHeapUnifiedMemory) { - _onHeapUnifiedMemory = executorMetrics.onHeapUnifiedMemory - updated = true - } - if (executorMetrics.offHeapUnifiedMemory > _offHeapUnifiedMemory) { - _offHeapUnifiedMemory = executorMetrics.offHeapUnifiedMemory - updated = true - } - if (executorMetrics.directMemory > _directMemory) { - _directMemory = executorMetrics.directMemory - updated = true - } - if (executorMetrics.mappedMemory > _mappedMemory) { - _mappedMemory = executorMetrics.mappedMemory - updated = true - } - updated } @@ -115,13 +53,18 @@ private[spark] class PeakExecutorMetrics { * values set. */ def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = { - if (_jvmUsedHeapMemory < 0) { + if (metrics(0) < 0) { None } else { - Some(new PeakMemoryMetrics(_jvmUsedHeapMemory, _jvmUsedNonHeapMemory, - _onHeapExecutionMemory, _offHeapExecutionMemory, _onHeapStorageMemory, - _offHeapStorageMemory, _onHeapUnifiedMemory, _offHeapUnifiedMemory, - _directMemory, _mappedMemory)) + val copy = new PeakMemoryMetrics + System.arraycopy(this.metrics, 0, copy.metrics, 0, this.metrics.length) + Some(copy) } } + + /** Clears/resets the saved peak values. */ + def reset(): Unit = { + (0 until metrics.length).foreach { idx => metrics(idx) = 0} + metrics(0) = -1 + } } 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 15f83213bb6e..aff0dcf7c9be 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 @@ -25,7 +25,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.JobExecutionStatus -import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.scheduler.MemoryTypes case class ApplicationInfo private[spark]( id: String, @@ -108,17 +108,10 @@ class MemoryMetrics private[spark]( val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) -class PeakMemoryMetrics private[spark]( - val jvmUsedHeapMemory: Long, - val jvmUsedNonHeapMemory: Long, - val onHeapExecutionMemory: Long, - val offHeapExecutionMemory: Long, - val onHeapStorageMemory: Long, - val offHeapStorageMemory: Long, - val onHeapUnifiedMemory: Long, - val offHeapUnifiedMemory: Long, - val directMemory: Long, - val mappedMemory: Long) +class PeakMemoryMetrics private[spark]() { + // TODO special json-ification + val metrics = new Array[Long](MemoryTypes.values().length) +} class JobData private[spark]( val jobId: Int, From 036148cdbe60b7ad7ff318260580896ad0da6cd0 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 23 May 2018 15:50:26 -0500 Subject: [PATCH 14/29] wip ... has both enum and non-enum version --- .../apache/spark/scheduler/MemoryTypes.java | 51 +++--------- .../org/apache/spark/executor/Executor.scala | 55 +++---------- .../spark/executor/ExecutorMetrics.scala | 26 +----- .../apache/spark/metrics/MetricGetter.scala | 80 +++++++++++++++++++ .../apache/spark/scheduler/DAGScheduler.scala | 9 +-- .../scheduler/EventLoggingListener.scala | 9 +-- .../spark/scheduler/PeakExecutorMetrics.scala | 6 +- 7 files changed, 112 insertions(+), 124 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala diff --git a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java index 876233916f0b..9897999d41b1 100644 --- a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java +++ b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java @@ -18,50 +18,21 @@ package org.apache.spark.scheduler; import org.apache.spark.executor.ExecutorMetrics; +import org.apache.spark.memory.MemoryManager; public enum MemoryTypes { - JvmUsedMemory{ + JvmUsedMemory { @Override - long get(ExecutorMetrics em) { - return em.jvmUsedMemory(); + long getValue(MemoryManager memoryManager) { + return 0; } }, - OnHeapExecutionMemory { - @Override - long get(ExecutorMetrics em) { - return em.onHeapExecutionMemory(); - } - }, - OffHeapExecutionMemory { - @Override - long get(ExecutorMetrics em) { - return em.offHeapExecutionMemory(); - } - }, - OnHeapStorageMemory { - @Override - long get(ExecutorMetrics em) { - return em.onHeapStorageMemory(); - } - }, - OffHeapStorageMemory { - @Override - long get(ExecutorMetrics em) { - return em.offHeapStorageMemory(); - } - }, - OnHeapUnifiedMemory { - @Override - long get(ExecutorMetrics em) { - return em.onHeapExecutionMemory() + em.onHeapStorageMemory(); - } - }, - OffHeapUnifiedMemory { - @Override - long get(ExecutorMetrics em) { - return em.offHeapExecutionMemory() + em.offHeapStorageMemory(); - } - }; + OnHeapExecutionMemory, + OffHeapExecutionMemory, + OnHeapStorageMemory, + OffHeapStorageMemory, + OnHeapUnifiedMemory, + OffHeapUnifiedMemory; - abstract long get(ExecutorMetrics em); + abstract long getValue(MemoryManager memoryManager); } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 7012a20c3f17..c9e2677fbc35 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -19,13 +19,12 @@ package org.apache.spark.executor import java.io.{File, NotSerializableException} import java.lang.Thread.UncaughtExceptionHandler -import java.lang.management.{BufferPoolMXBean, ManagementFactory} +import java.lang.management.ManagementFactory import java.net.{URI, URL} import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent._ import javax.annotation.concurrent.GuardedBy -import javax.management.ObjectName import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -38,8 +37,9 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.memory.{MemoryManager, SparkOutOfMemoryError, TaskMemoryManager} +import org.apache.spark.metrics.MetricGetter import org.apache.spark.rpc.RpcTimeout -import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription} +import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ @@ -72,12 +72,6 @@ private[spark] class Executor( private val conf = env.conf - // BufferPoolMXBean for direct memory - private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME) - - // BufferPoolMXBean for mapped memory - private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME) - // No ip or host:port - just hostname Utils.checkHost(executorHostname) // must not have port specified. @@ -795,8 +789,7 @@ private[spark] class Executor( val curGCTime = computeTotalGcTime() // get executor level memory metrics - val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager, - directBufferPool, mappedBufferPool) + val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager) for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { @@ -835,42 +828,14 @@ private[spark] object Executor { // used instead. val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties] - val DIRECT_BUFFER_POOL_NAME = "direct" - val MAPPED_BUFFER_POOL_NAME = "mapped" - - /** Get the BufferPoolMXBean for the specified buffer pool. */ - def getBufferPool(pool: String): BufferPoolMXBean = { - val name = new ObjectName("java.nio:type=BufferPool,name=" + pool) - ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, - name.toString, classOf[BufferPoolMXBean]) - } - /** * Get the current executor level memory metrics. - * - * @param memoryManager the memory manager - * @param direct the direct memory buffer pool - * @param mapped the mapped memory buffer pool - * @return the executor memory metrics */ - def getCurrentExecutorMetrics( - memoryManager: MemoryManager, - direct: BufferPoolMXBean, - mapped: BufferPoolMXBean) : ExecutorMetrics = { - val onHeapExecutionMemoryUsed = memoryManager.onHeapExecutionMemoryUsed - val offHeapExecutionMemoryUsed = memoryManager.offHeapExecutionMemoryUsed - val onHeapStorageMemoryUsed = memoryManager.onHeapStorageMemoryUsed - val offHeapStorageMemoryUsed = memoryManager.offHeapStorageMemoryUsed - new ExecutorMetrics(System.currentTimeMillis(), - ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(), - ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed(), - onHeapExecutionMemoryUsed, - offHeapExecutionMemoryUsed, - onHeapStorageMemoryUsed, - offHeapStorageMemoryUsed, - onHeapExecutionMemoryUsed + onHeapStorageMemoryUsed, // on heap unified memory - offHeapExecutionMemoryUsed + offHeapStorageMemoryUsed, // off heap unified memory - direct.getMemoryUsed, - mapped.getMemoryUsed) + def getCurrentExecutorMetrics(memoryManager: MemoryManager): ExecutorMetrics = { + val metrics = new ExecutorMetrics(System.currentTimeMillis()) + MetricGetter.idxAndValues.foreach { case (idx, metric) => + metrics.metrics(idx) = metric.getMetricValue(memoryManager) + } + metrics } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 6d35a1c682e2..ce9ad66bafb9 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -18,6 +18,7 @@ package org.apache.spark.executor import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.scheduler.MemoryTypes /** * :: DeveloperApi :: @@ -28,27 +29,8 @@ import org.apache.spark.annotation.DeveloperApi * * @param timestamp the time the metrics were collected, or -1 for Spark history * log events which are logged when a stage has completed - * @param jvmUsedHeapMemory the amount of JVM used heap memory for the executor - * @param jvmUsedNonHeapMemory the amount of JVM used non-heap memory for the executor - * @param onHeapExecutionMemory the amount of on heap execution memory used - * @param offHeapExecutionMemory the amount of off heap execution memory used - * @param onHeapStorageMemory the amount of on heap storage memory used - * @param offHeapStorageMemory the amount of off heap storage memory used - * @param onHeapUnifiedMemory the amount of on heap unified region memory used - * @param offHeapUnifiedMemory the amount of off heap unified region memory used - * @param directMemory the amount of direct memory used - * @param mappedMemory the amount of mapped memory used */ @DeveloperApi -class ExecutorMetrics private[spark] ( - val timestamp: Long, - val jvmUsedHeapMemory: Long, - val jvmUsedNonHeapMemory: Long, - val onHeapExecutionMemory: Long, - val offHeapExecutionMemory: Long, - val onHeapStorageMemory: Long, - val offHeapStorageMemory: Long, - val onHeapUnifiedMemory: Long, - val offHeapUnifiedMemory: Long, - val directMemory: Long, - val mappedMemory: Long) extends Serializable +class ExecutorMetrics private[spark] (val timestamp: Long) extends Serializable { + val metrics = new Array[Long](MemoryTypes.values().length) +} diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala new file mode 100644 index 000000000000..d32ea9df77f7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala @@ -0,0 +1,80 @@ +/* + * 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.metrics + +import java.lang.management.{BufferPoolMXBean, ManagementFactory} +import javax.management.ObjectName + +import org.apache.spark.memory.MemoryManager + +sealed trait MetricGetter { + def getMetricValue(memoryManager: MemoryManager): Long +} + +abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter { + override def getMetricValue(memoryManager: MemoryManager): Long = { + f(memoryManager) + } +} + +abstract class MBeanMetricGetter(mBeanName: String) extends MetricGetter { + private val name = new ObjectName(mBeanName) + val bean = ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, + name.toString, classOf[BufferPoolMXBean]) + + override def getMetricValue(memoryManager: MemoryManager): Long = { + bean.getMemoryUsed + } +} + +case object JVMHeapMemory extends MetricGetter { + override def getMetricValue(memoryManager: MemoryManager): Long = { + ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed() + } +} + +case object JVMOffHeapMemory extends MetricGetter { + override def getMetricValue(memoryManager: MemoryManager): Long = { + ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed() + } +} + +case object OnHeapExecution extends MemoryManagerMetricGetter(_.onHeapExecutionMemoryUsed) + +case object OffHeapExecution extends MemoryManagerMetricGetter(_.offHeapExecutionMemoryUsed) + +case object OnHeapStorage extends MemoryManagerMetricGetter(_.onHeapStorageMemoryUsed) + +case object OffHeapStorage extends MemoryManagerMetricGetter(_.offHeapStorageMemoryUsed) + +case object DirectPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=direct") +case object MappedPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=mapped") + +object MetricGetter { + val values = IndexedSeq( + JVMHeapMemory, + JVMOffHeapMemory, + OnHeapExecution, + OffHeapExecution, + OnHeapStorage, + OffHeapStorage, + DirectPoolMemory, + MappedPoolMemory + ) + + val idxAndValues = values.zipWithIndex.map(_.swap) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b7dc716ec1c0..78ac6f0b2ff7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -214,12 +214,6 @@ class DAGScheduler( private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, "driver-heartbeater", sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) - /** BufferPoolMXBean for direct memory */ - private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME) - - /** BufferPoolMXBean for mapped memory */ - private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME) - /** * Called by the TaskSetManager to report task's starting. */ @@ -1770,8 +1764,7 @@ class DAGScheduler( /** Reports heartbeat metrics for the driver. */ private def reportHeartBeat(): Unit = { // get driver memory metrics - val driverUpdates = Executor.getCurrentExecutorMetrics( - sc.env.memoryManager, directBufferPool, mappedBufferPool) + val driverUpdates = Executor.getCurrentExecutorMetrics(sc.env.memoryManager) val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, Some(driverUpdates))) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index a795186f20b2..baf7127a8392 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -197,12 +197,9 @@ logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecut executorMap.foreach { executorEntry => { for ((executorId, peakExecutorMetrics) <- executorEntry) { - val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory, - peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory, - peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory, - peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory, - peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory, - peakExecutorMetrics.mappedMemory) + val executorMetrics = new ExecutorMetrics(-1) + System.arraycopy(peakExecutorMetrics.metrics, 0, executorMetrics.metrics, 0, + peakExecutorMetrics.metrics.size) val executorUpdate = new SparkListenerExecutorMetricsUpdate( executorId, accumUpdates, Some(executorMetrics)) logEvent(executorUpdate) diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index a91aabde9306..e40474eb4709 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -39,10 +39,10 @@ private[spark] class PeakExecutorMetrics { var updated: Boolean = false (0 until MemoryTypes.values().length).foreach { metricIdx => - val metricVal = MemoryTypes.values()(metricIdx).get(executorMetrics) - if (metricVal > metrics(metricIdx)) { + val newVal = executorMetrics.metrics(metricIdx) + if ( newVal > metrics(metricIdx)) { updated = true - metrics(metricIdx) = metricVal + metrics(metricIdx) = newVal } } updated From 91fb1db09504fc4386477ab51221d28240c3c901 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 23 May 2018 16:23:44 -0500 Subject: [PATCH 15/29] case objects, mostly complete --- .../apache/spark/scheduler/MemoryTypes.java | 38 ------------------- .../spark/executor/ExecutorMetrics.scala | 4 +- .../apache/spark/metrics/MetricGetter.scala | 4 +- .../spark/scheduler/PeakExecutorMetrics.scala | 5 ++- .../org/apache/spark/status/api/v1/api.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 38 +++++++------------ .../scheduler/EventLoggingListenerSuite.scala | 15 ++------ .../apache/spark/util/JsonProtocolSuite.scala | 10 ++--- 8 files changed, 30 insertions(+), 88 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java diff --git a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java b/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java deleted file mode 100644 index 9897999d41b1..000000000000 --- a/core/src/main/java/org/apache/spark/scheduler/MemoryTypes.java +++ /dev/null @@ -1,38 +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.scheduler; - -import org.apache.spark.executor.ExecutorMetrics; -import org.apache.spark.memory.MemoryManager; - -public enum MemoryTypes { - JvmUsedMemory { - @Override - long getValue(MemoryManager memoryManager) { - return 0; - } - }, - OnHeapExecutionMemory, - OffHeapExecutionMemory, - OnHeapStorageMemory, - OffHeapStorageMemory, - OnHeapUnifiedMemory, - OffHeapUnifiedMemory; - - abstract long getValue(MemoryManager memoryManager); -} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index ce9ad66bafb9..663aecfbebd3 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -18,7 +18,7 @@ package org.apache.spark.executor import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.scheduler.MemoryTypes +import org.apache.spark.metrics.MetricGetter /** * :: DeveloperApi :: @@ -32,5 +32,5 @@ import org.apache.spark.scheduler.MemoryTypes */ @DeveloperApi class ExecutorMetrics private[spark] (val timestamp: Long) extends Serializable { - val metrics = new Array[Long](MemoryTypes.values().length) + val metrics = new Array[Long](MetricGetter.values.length) } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala index d32ea9df77f7..53c87a1261e1 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala @@ -23,6 +23,7 @@ import org.apache.spark.memory.MemoryManager sealed trait MetricGetter { def getMetricValue(memoryManager: MemoryManager): Long + val name = getClass().getName().stripSuffix("$") } abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter { @@ -32,9 +33,8 @@ abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends Metri } abstract class MBeanMetricGetter(mBeanName: String) extends MetricGetter { - private val name = new ObjectName(mBeanName) val bean = ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, - name.toString, classOf[BufferPoolMXBean]) + new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean]) override def getMetricValue(memoryManager: MemoryManager): Long = { bean.getMemoryUsed diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index e40474eb4709..83bb39cf3c3e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.metrics.MetricGetter import org.apache.spark.status.api.v1.PeakMemoryMetrics /** @@ -25,7 +26,7 @@ import org.apache.spark.status.api.v1.PeakMemoryMetrics * values have been recorded yet. */ private[spark] class PeakExecutorMetrics { - val metrics = new Array[Long](MemoryTypes.values().length) + val metrics = new Array[Long](MetricGetter.values.length) metrics(0) = -1 /** @@ -38,7 +39,7 @@ private[spark] class PeakExecutorMetrics { def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { var updated: Boolean = false - (0 until MemoryTypes.values().length).foreach { metricIdx => + (0 until MetricGetter.values.length).foreach { metricIdx => val newVal = executorMetrics.metrics(metricIdx) if ( newVal > metrics(metricIdx)) { updated = true 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 aff0dcf7c9be..3d6360383683 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 @@ -25,7 +25,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties import com.fasterxml.jackson.databind.annotation.JsonDeserialize import org.apache.spark.JobExecutionStatus -import org.apache.spark.scheduler.MemoryTypes +import org.apache.spark.metrics.MetricGetter case class ApplicationInfo private[spark]( id: String, @@ -110,7 +110,7 @@ class MemoryMetrics private[spark]( class PeakMemoryMetrics private[spark]() { // TODO special json-ification - val metrics = new Array[Long](MemoryTypes.values().length) + val metrics = new Array[Long](MetricGetter.values.length) } class JobData private[spark]( 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 8a36a93c858e..0e753778fc81 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -31,6 +31,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark._ import org.apache.spark.executor._ +import org.apache.spark.metrics.MetricGetter import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -389,17 +390,12 @@ private[spark] object JsonProtocol { * @return the JSON representation */ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { - ("Timestamp" -> executorMetrics.timestamp) ~ - ("JVM Used Heap Memory" -> executorMetrics.jvmUsedHeapMemory) ~ - ("JVM Used Nonheap Memory" -> executorMetrics.jvmUsedNonHeapMemory) ~ - ("Onheap Execution Memory" -> executorMetrics.onHeapExecutionMemory) ~ - ("Offheap Execution Memory" -> executorMetrics.offHeapExecutionMemory) ~ - ("Onheap Storage Memory" -> executorMetrics.onHeapStorageMemory) ~ - ("Offheap Storage Memory" -> executorMetrics.offHeapStorageMemory) ~ - ("Onheap Unified Memory" -> executorMetrics.onHeapUnifiedMemory) ~ - ("Offheap Unified Memory" -> executorMetrics.offHeapUnifiedMemory) ~ - ("Direct Memory" -> executorMetrics.directMemory) ~ - ("Mapped Memory" -> executorMetrics.mappedMemory) + val metrics = MetricGetter.idxAndValues.map { case (idx, metric) => + JField(metric.name, executorMetrics.metrics(idx)) + } + JObject( + (Seq(JField("Timestamp", executorMetrics.timestamp)) ++ metrics): _* + ) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -616,20 +612,12 @@ private[spark] object JsonProtocol { */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val timeStamp = (json \ "Timestamp").extract[Long] - val jvmUsedHeapMemory = (json \ "JVM Used Heap Memory").extract[Long] - val jvmUsedNonHeapMemory = (json \ "JVM Used Nonheap Memory").extract[Long] - val onHeapExecutionMemory = (json \ "Onheap Execution Memory").extract[Long] - val offHeapExecutionMemory = (json \ "Offheap Execution Memory").extract[Long] - val onHeapStorageMemory = (json \ "Onheap Storage Memory").extract[Long] - val offHeapStorageMemory = (json \ "Offheap Storage Memory").extract[Long] - val onHeapUnifiedMemory = (json \ "Onheap Unified Memory").extract[Long] - val offHeapUnifiedMemory = (json \ "Offheap Unified Memory").extract[Long] - val directMemory = (json \ "Direct Memory").extract[Long] - val mappedMemory = (json \ "Mapped Memory").extract[Long] - new ExecutorMetrics(timeStamp, jvmUsedHeapMemory, jvmUsedNonHeapMemory, - onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, - offHeapStorageMemory, onHeapUnifiedMemory, offHeapUnifiedMemory, directMemory, - mappedMemory) + val metrics = new ExecutorMetrics(timeStamp) + MetricGetter.idxAndValues.foreach { case (idx, metric) => + val metricValue = (json \ metric.name).extract[Long] + metrics.metrics(idx) = metricValue + } + metrics } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { 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 6b906a88183c..f6ad8a6f16d8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.io._ -import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.metrics.{MetricGetter, MetricsSystem} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} @@ -412,16 +412,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit (executorMetrics1, executorMetrics2) match { case (Some(e1), Some(e2)) => assert(e1.timestamp === e2.timestamp) - assert(e1.jvmUsedHeapMemory === e2.jvmUsedHeapMemory) - assert(e1.jvmUsedNonHeapMemory === e2.jvmUsedNonHeapMemory) - assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory) - assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory) - assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory) - assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory) - assert(e1.onHeapUnifiedMemory === e2.onHeapUnifiedMemory) - assert(e1.offHeapUnifiedMemory === e2.offHeapUnifiedMemory) - assert(e1.directMemory === e2.directMemory) - assert(e1.mappedMemory === e2.mappedMemory) + (0 until MetricGetter.values.length).foreach { idx => + assert(e1.metrics(idx) === e2.metrics(idx)) + } case (None, None) => case _ => assert(false) 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 afe11afe3e5c..4f3b412dbd43 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -30,6 +30,7 @@ import org.scalatest.exceptions.TestFailedException import org.apache.spark._ import org.apache.spark.executor._ +import org.apache.spark.metrics.MetricGetter import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -686,12 +687,9 @@ private[spark] object JsonProtocolSuite extends Assertions { (metrics1, metrics2) match { case (Some(m1), Some(m2)) => assert(m1.timestamp === m2.timestamp) - assert(m1.jvmUsedHeapMemory === m2.jvmUsedHeapMemory) - assert(m1.jvmUsedNonHeapMemory === m2.jvmUsedNonHeapMemory) - assert(m1.onHeapExecutionMemory === m2.onHeapExecutionMemory) - assert(m1.offHeapExecutionMemory === m2.offHeapExecutionMemory) - assert(m1.onHeapStorageMemory === m2.onHeapStorageMemory) - assert(m1.offHeapStorageMemory === m2.offHeapStorageMemory) + (0 until MetricGetter.values.length).foreach { idx => + assert(m1.metrics(idx) === m2.metrics(idx)) + } case (None, None) => case _ => assert(false) From 2d8894a91f4a0dacd49114dc74cc97b7c9426879 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Sun, 10 Jun 2018 14:34:19 -0700 Subject: [PATCH 16/29] Address comments (move heartbeater from DAGScheduler to SparkContext, move logic for getting metrics to Heartbeater), and modifiy tests for the new ExecutorMetrics format. --- .../scala/org/apache/spark/Heartbeater.scala | 17 ++- .../scala/org/apache/spark/SparkContext.scala | 25 ++++ .../org/apache/spark/executor/Executor.scala | 20 +-- .../spark/executor/ExecutorMetrics.scala | 11 +- .../apache/spark/metrics/MetricGetter.scala | 26 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 20 +-- .../scheduler/EventLoggingListener.scala | 18 ++- .../spark/scheduler/PeakExecutorMetrics.scala | 15 --- .../org/apache/spark/status/LiveEntity.scala | 2 +- .../org/apache/spark/status/api/v1/api.scala | 45 ++++++- .../org/apache/spark/util/JsonProtocol.scala | 12 +- ...ith_executor_metrics_json_expectation.json | 100 +++++++-------- .../application_1506645932520_24630151 | 20 +-- .../apache/spark/HeartbeatReceiverSuite.scala | 4 +- .../scheduler/EventLoggingListenerSuite.scala | 114 +++++++++--------- .../spark/status/AppStatusListenerSuite.scala | 47 +++----- .../apache/spark/util/JsonProtocolSuite.scala | 26 ++-- 17 files changed, 278 insertions(+), 244 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 2c89cc14189d..ca644c1a2b36 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -19,17 +19,26 @@ package org.apache.spark import java.util.concurrent.TimeUnit +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.internal.Logging +import org.apache.spark.memory.MemoryManager +import org.apache.spark.metrics.MetricGetter import org.apache.spark.util.{ThreadUtils, Utils} /** * Creates a heartbeat thread which will call the specified reportHeartbeat function at * intervals of intervalMs. * + * @param memoryManager the memory manager for execution and storage memory. * @param reportHeartbeat the heartbeat reporting function to call. * @param name the thread name for the heartbeater. * @param intervalMs the interval between heartbeats. */ -private[spark] class Heartbeater(reportHeartbeat: () => Unit, name: String, intervalMs: Long) { +private[spark] class Heartbeater( + memoryManager: MemoryManager, + reportHeartbeat: () => Unit, + name: String, + intervalMs: Long) extends Logging { // Executor for the heartbeat task private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor(name) @@ -49,5 +58,11 @@ private[spark] class Heartbeater(reportHeartbeat: () => Unit, name: String, inte heartbeater.shutdown() heartbeater.awaitTermination(10, TimeUnit.SECONDS) } + + /** Get the current metrics. */ + def getCurrentMetrics(): ExecutorMetrics = { + new ExecutorMetrics(System.currentTimeMillis(), + MetricGetter.values.map(_.getMetricValue(memoryManager)).toArray) + } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5e8595603cc9..5156e11a01ef 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -213,6 +213,7 @@ class SparkContext(config: SparkConf) extends Logging { private var _files: Seq[String] = _ private var _shutdownHookRef: AnyRef = _ private var _statusStore: AppStatusStore = _ + private var _heartbeater: Heartbeater = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -304,6 +305,11 @@ class SparkContext(config: SparkConf) extends Logging { _dagScheduler = ds } + private[spark] def heartbeater: Heartbeater = _heartbeater + private[spark] def heartbeater_=(hb: Heartbeater): Unit = { + _heartbeater = hb + } + /** * A unique identifier for the Spark application. * Its format depends on the scheduler implementation. @@ -496,6 +502,11 @@ class SparkContext(config: SparkConf) extends Logging { _dagScheduler = new DAGScheduler(this) _heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet) + // create and start the heartbeater for collecting memory metrics + _heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, "driver-heartbeater", + conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + _heartbeater.start() + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's // constructor _taskScheduler.start() @@ -1922,6 +1933,12 @@ class SparkContext(config: SparkConf) extends Logging { Utils.tryLogNonFatalError { _eventLogger.foreach(_.stop()) } + if(_heartbeater != null) { + Utils.tryLogNonFatalError { + _heartbeater.stop() + } + _heartbeater = null + } if (_dagScheduler != null) { Utils.tryLogNonFatalError { _dagScheduler.stop() @@ -2398,6 +2415,14 @@ class SparkContext(config: SparkConf) extends Logging { } } + /** Reports heartbeat metrics for the driver. */ + private def reportHeartBeat(): Unit = { + val driverUpdates = _heartbeater.getCurrentMetrics() + val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) + listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, + Some(driverUpdates))) + } + // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having finished construction. // NOTE: this must be placed at the end of the SparkContext constructor. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c9e2677fbc35..6c67e640b338 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -36,8 +36,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.memory.{MemoryManager, SparkOutOfMemoryError, TaskMemoryManager} -import org.apache.spark.metrics.MetricGetter +import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException @@ -149,8 +148,8 @@ private[spark] class Executor( private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] // Executor for the heartbeat task. - private val heartbeater = new Heartbeater(reportHeartBeat, "executor-heartbeater", - conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + private val heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, + "executor-heartbeater", conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) // must be initialized before running startDriverHeartbeat() private val heartbeatReceiverRef = @@ -789,7 +788,7 @@ private[spark] class Executor( val curGCTime = computeTotalGcTime() // get executor level memory metrics - val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager) + val executorUpdates = heartbeater.getCurrentMetrics() for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { @@ -827,15 +826,4 @@ private[spark] object Executor { // task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be // used instead. val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties] - - /** - * Get the current executor level memory metrics. - */ - def getCurrentExecutorMetrics(memoryManager: MemoryManager): ExecutorMetrics = { - val metrics = new ExecutorMetrics(System.currentTimeMillis()) - MetricGetter.idxAndValues.foreach { case (idx, metric) => - metrics.metrics(idx) = metric.getMetricValue(memoryManager) - } - metrics - } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 663aecfbebd3..9305249ccb34 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -29,8 +29,15 @@ import org.apache.spark.metrics.MetricGetter * * @param timestamp the time the metrics were collected, or -1 for Spark history * log events which are logged when a stage has completed + * @param metrics the array of executor metrics values, order and elements as + * specified in MetricGetter */ @DeveloperApi -class ExecutorMetrics private[spark] (val timestamp: Long) extends Serializable { - val metrics = new Array[Long](MetricGetter.values.length) +class ExecutorMetrics private[spark] ( + val timestamp: Long, + val metrics: Array[Long]) extends Serializable { + if (metrics.length != MetricGetter.values.length) { + throw new IllegalArgumentException("invalid metrics length " + metrics.length + + " does not equal expected length " + MetricGetter.values.length) + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala index 53c87a1261e1..27ef6d94744a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala @@ -23,7 +23,7 @@ import org.apache.spark.memory.MemoryManager sealed trait MetricGetter { def getMetricValue(memoryManager: MemoryManager): Long - val name = getClass().getName().stripSuffix("$") + val name = getClass().getName().stripSuffix("$").split("""\.""").last } abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter { @@ -53,13 +53,19 @@ case object JVMOffHeapMemory extends MetricGetter { } } -case object OnHeapExecution extends MemoryManagerMetricGetter(_.onHeapExecutionMemoryUsed) +case object OnHeapExecutionMemory extends MemoryManagerMetricGetter(_.onHeapExecutionMemoryUsed) -case object OffHeapExecution extends MemoryManagerMetricGetter(_.offHeapExecutionMemoryUsed) +case object OffHeapExecutionMemory extends MemoryManagerMetricGetter(_.offHeapExecutionMemoryUsed) -case object OnHeapStorage extends MemoryManagerMetricGetter(_.onHeapStorageMemoryUsed) +case object OnHeapStorageMemory extends MemoryManagerMetricGetter(_.onHeapStorageMemoryUsed) -case object OffHeapStorage extends MemoryManagerMetricGetter(_.offHeapStorageMemoryUsed) +case object OffHeapStorageMemory extends MemoryManagerMetricGetter(_.offHeapStorageMemoryUsed) + +case object OnHeapUnifiedMemory extends MemoryManagerMetricGetter( + (m => m.onHeapExecutionMemoryUsed + m.onHeapStorageMemoryUsed)) + +case object OffHeapUnifiedMemory extends MemoryManagerMetricGetter( + (m => m.offHeapExecutionMemoryUsed + m.offHeapStorageMemoryUsed)) case object DirectPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=direct") case object MappedPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=mapped") @@ -68,10 +74,12 @@ object MetricGetter { val values = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, - OnHeapExecution, - OffHeapExecution, - OnHeapStorage, - OffHeapStorage, + OnHeapExecutionMemory, + OffHeapExecutionMemory, + OnHeapStorageMemory, + OffHeapStorageMemory, + OnHeapUnifiedMemory, + OffHeapUnifiedMemory, DirectPoolMemory, MappedPoolMemory ) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 78ac6f0b2ff7..ee99d7aaf056 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io.NotSerializableException -import java.lang.management.ManagementFactory import java.util.Properties import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger @@ -35,7 +34,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.{Executor, ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils @@ -210,10 +209,6 @@ class DAGScheduler( private[spark] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) taskScheduler.setDAGScheduler(this) - /** driver heartbeat for collecting metrics */ - private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, "driver-heartbeater", - sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) - /** * Called by the TaskSetManager to report task's starting. */ @@ -1758,20 +1753,9 @@ class DAGScheduler( messageScheduler.shutdownNow() eventProcessLoop.stop() taskScheduler.stop() - heartbeater.stop() - } - - /** Reports heartbeat metrics for the driver. */ - private def reportHeartBeat(): Unit = { - // get driver memory metrics - val driverUpdates = Executor.getCurrentExecutorMetrics(sc.env.memoryManager) - val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) - listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, - Some(driverUpdates))) - } + } eventProcessLoop.start() - heartbeater.start() } private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index baf7127a8392..f0da348a7ec0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -24,7 +24,7 @@ import java.util.EnumSet import java.util.Locale import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} @@ -84,7 +84,7 @@ private[spark] class EventLoggingListener( private val compressionCodecName = compressionCodec.map { c => CompressionCodec.getShortName(c.getClass.getName) } -logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecutorMetricsUpdates) + // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None @@ -97,8 +97,7 @@ logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecut private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) // map of live stages, to peak executor metrics for the stage - private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int), - mutable.HashMap[String, PeakExecutorMetrics]]() + private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, PeakExecutorMetrics]]() /** * Creates the log file in the configured log directory. @@ -167,7 +166,7 @@ logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecut if (shouldLogExecutorMetricsUpdates) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), - new mutable.HashMap[String, PeakExecutorMetrics]()) + new HashMap[String, PeakExecutorMetrics]()) } } @@ -190,16 +189,15 @@ logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecut liveStageExecutorMetrics.remove((event.stageInfo.stageId, attemptId)) } - // log the peak executor metrics for the stage, for each executor + // log the peak executor metrics for the stage, for each live executor, + // whether or not the executor is running tasks for the stage val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() val executorMap = liveStageExecutorMetrics.remove( (event.stageInfo.stageId, event.stageInfo.attemptNumber())) executorMap.foreach { executorEntry => { for ((executorId, peakExecutorMetrics) <- executorEntry) { - val executorMetrics = new ExecutorMetrics(-1) - System.arraycopy(peakExecutorMetrics.metrics, 0, executorMetrics.metrics, 0, - peakExecutorMetrics.metrics.size) + val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.metrics) val executorUpdate = new SparkListenerExecutorMetricsUpdate( executorId, accumUpdates, Some(executorMetrics)) logEvent(executorUpdate) @@ -346,7 +344,7 @@ private[spark] object EventLoggingListener extends Logging { private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) // A cache for compression codecs to avoid creating the same codec many times - private val codecMap = new mutable.HashMap[String, CompressionCodec] + private val codecMap = new HashMap[String, CompressionCodec] /** * Write metadata about an event log to the given stream. diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index 83bb39cf3c3e..b7c279e63b45 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.metrics.MetricGetter -import org.apache.spark.status.api.v1.PeakMemoryMetrics /** * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no @@ -49,20 +48,6 @@ private[spark] class PeakExecutorMetrics { updated } - /** - * @return None if no peak metrics have been recorded, else PeakMemoryMetrics with the peak - * values set. - */ - def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = { - if (metrics(0) < 0) { - None - } else { - val copy = new PeakMemoryMetrics - System.arraycopy(this.metrics, 0, copy.metrics, 0, this.metrics.length) - Some(copy) - } - } - /** Clears/resets the saved peak values. */ def reset(): Unit = { (0 until metrics.length).foreach { idx => metrics(idx) = 0} 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 c296f55210f2..c747ce66b78c 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -306,7 +306,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE executorLogs, memoryMetrics, blacklistedInStages, - peakExecutorMetrics.getPeakMemoryMetrics) + if (peakExecutorMetrics.metrics(0) == -1) None else Some(peakExecutorMetrics.metrics)) 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 3d6360383683..d36588ae40da 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 @@ -22,7 +22,9 @@ import java.util.Date import scala.xml.{NodeSeq, Text} import com.fasterxml.jackson.annotation.JsonIgnoreProperties -import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import com.fasterxml.jackson.core.{JsonGenerator, JsonParser} +import com.fasterxml.jackson.databind.{DeserializationContext, JsonDeserializer, JsonSerializer, SerializerProvider} +import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize} import org.apache.spark.JobExecutionStatus import org.apache.spark.metrics.MetricGetter @@ -100,7 +102,9 @@ class ExecutorSummary private[spark]( val executorLogs: Map[String, String], val memoryMetrics: Option[MemoryMetrics], val blacklistedInStages: Set[Int], - val peakMemoryMetrics: Option[PeakMemoryMetrics]) + @JsonSerialize(using = classOf[PeakMemoryMetricsSerializer]) + @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer]) + val peakMemoryMetrics: Option[Array[Long]]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, @@ -108,9 +112,40 @@ class MemoryMetrics private[spark]( val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) -class PeakMemoryMetrics private[spark]() { - // TODO special json-ification - val metrics = new Array[Long](MetricGetter.values.length) +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */ +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] { + override def deserialize( + jsonParser: JsonParser, + deserializationContext: DeserializationContext): Option[Array[Long]] = { + val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]]) + metricsMap match { + case Some(metrics) => + Some(MetricGetter.values.map { m => + metrics.getOrElse (m.name, 0L) match { + case intVal: Int => intVal.toLong + case longVal: Long => longVal + } + }.toArray) + case None => None + } + } +} + +/** serializer for peakMemoryMetrics: convert array to map with metric name as key */ +class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] { + override def serialize( + metrics: Option[Array[Long]], + jsonGenerator: JsonGenerator, + serializerProvider: SerializerProvider): Unit = { + metrics match { + case Some(m) => + val metricsMap = (0 until MetricGetter.values.length).map { idx => + MetricGetter.values (idx).name -> m(idx) + }.toMap + jsonGenerator.writeObject (metricsMap) + case None => + } + } } class JobData private[spark]( 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 0e753778fc81..e42fc0d94d03 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -612,12 +612,12 @@ private[spark] object JsonProtocol { */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val timeStamp = (json \ "Timestamp").extract[Long] - val metrics = new ExecutorMetrics(timeStamp) - MetricGetter.idxAndValues.foreach { case (idx, metric) => - val metricValue = (json \ metric.name).extract[Long] - metrics.metrics(idx) = metricValue - } - metrics + + val metrics = + MetricGetter.values.map {metric => + val metricVal = (json \ metric.name).extract[Long] + metricVal} + new ExecutorMetrics(timeStamp, metrics.toArray) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { 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 ac1bb97b157f..9bf2086cc8e7 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 @@ -28,16 +28,16 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 629553808, - "jvmUsedNonHeapMemory" : 205304696, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 905801, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 905801, - "offHeapUnifiedMemory" : 0, - "directMemory" : 397602, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 905801, + "JVMOffHeapMemory" : 205304696, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 905801, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 397602, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 629553808, + "OffHeapStorageMemory" : 0 } }, { "id" : "7", @@ -168,16 +168,16 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 518613056, - "jvmUsedNonHeapMemory" : 95657456, - "onHeapExecutionMemory" : 37748736, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 63104457, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 100853193, - "offHeapUnifiedMemory" : 0, - "directMemory" : 126261, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 63104457, + "JVMOffHeapMemory" : 95657456, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 100853193, + "OnHeapExecutionMemory" : 37748736, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 126261, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 518613056, + "OffHeapStorageMemory" : 0 } }, { "id" : "3", @@ -212,16 +212,16 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 726805712, - "jvmUsedNonHeapMemory" : 90709624, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 69535048, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 69535048, - "offHeapUnifiedMemory" : 0, - "directMemory" : 87796, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 69535048, + "JVMOffHeapMemory" : 90709624, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 69535048, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 87796, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 726805712, + "OffHeapStorageMemory" : 0 } }, { "id" : "2", @@ -256,16 +256,16 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 595946552, - "jvmUsedNonHeapMemory" : 91208368, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 58468944, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 58468944, - "offHeapUnifiedMemory" : 0, - "directMemory" : 87796, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 58468944, + "JVMOffHeapMemory" : 91208368, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 58468944, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 87796, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 595946552, + "OffHeapStorageMemory" : 0 } }, { "id" : "1", @@ -300,15 +300,15 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "jvmUsedHeapMemory" : 755008624, - "jvmUsedNonHeapMemory" : 100519936, - "onHeapExecutionMemory" : 0, - "offHeapExecutionMemory" : 0, - "onHeapStorageMemory" : 47962185, - "offHeapStorageMemory" : 0, - "onHeapUnifiedMemory" : 47962185, - "offHeapUnifiedMemory" : 0, - "directMemory" : 98230, - "mappedMemory" : 0 + "OnHeapStorageMemory" : 47962185, + "JVMOffHeapMemory" : 100519936, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 47962185, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 98230, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 755008624, + "OffHeapStorageMemory" : 0 } } ] diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 index b3424f9582c0..8d132d57ef6e 100644 --- a/core/src/test/resources/spark-events/application_1506645932520_24630151 +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -33,11 +33,11 @@ {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144246,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"1920974","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"3562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"38","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1813","Value":"1812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"195602","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"3563","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1558","Value":"1557","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":3563,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":36845111,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":27318908,"Value":27318908,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":3562,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":349287,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":33,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1498974375,"Value":1498974375,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1922,"Value":1922,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":49547405,"Value":49547405,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":56,"Executor Deserialize CPU Time":49547405,"Executor Run Time":1922,"Executor CPU Time":1498974375,"Result Size":2394,"JVM GC Time":33,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":27318908,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144444,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"23365","Value":"63634","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":105575962,"Value":301246724,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":326,"Value":1107,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11931694025,"Value":31991331624,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13454,"Value":36578,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":531799977,"Value":1633060096,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":594,"Value":1930,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":594,"Executor Deserialize CPU Time":531799977,"Executor Run Time":13454,"Executor CPU Time":11931694025,"Result Size":1856,"JVM GC Time":326,"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":22950296,"Shuffle Write Time":105575962,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144840,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"24254","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":106963069,"Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":342,"Value":1449,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":12267596062,"Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13858,"Value":50436,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":519573839,"Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":573,"Value":2503,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":573,"Executor Deserialize CPU Time":519573839,"Executor Run Time":13858,"Executor CPU Time":12267596062,"Result Size":1856,"JVM GC Time":342,"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":31362123,"Shuffle Write Time":106963069,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":592412824,"JVM Used Nonheap Memory":202907152,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":905801,"Offheap Storage Memory":0,"Onheap Unified Memory":905801,"Offheap Unified Memory":0,"Direct Memory":355389,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":523121272,"JVM Used Nonheap Memory":88280720,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":52050147,"Offheap Storage Memory":0,"Onheap Unified Memory":52050147,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":214174608,"JVM Used Nonheap Memory":91548704,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47399168,"Offheap Storage Memory":0,"Onheap Unified Memory":47399168,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":518613056,"JVM Used Nonheap Memory":95657456,"Onheap Execution Memory":37748736,"Offheap Execution Memory":0,"Onheap Storage Memory":63104457,"Offheap Storage Memory":0,"Onheap Unified Memory":100853193,"Offheap Unified Memory":0,"Direct Memory":126261,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":726805712,"JVM Used Nonheap Memory":90709624,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69535048,"Offheap Storage Memory":0,"Onheap Unified Memory":69535048,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":592412824,"JVMOffHeapMemory":202907152,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":355389,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":523121272,"JVMOffHeapMemory":88280720,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":52050147,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":52050147,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":214174608,"JVMOffHeapMemory":91548704,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47399168,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47399168,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Completion Time":1524182144852,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":2503,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":50436,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":1449,"Internal":true,"Count Failed Values":true}]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145327,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"49","Value":"86","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"2002","Value":"3814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1755","Value":"3312","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":32035583,"Value":59354491,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4788,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1785119941,"Value":3284094316,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":2182,"Value":4104,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":71500541,"Value":121047946,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":136,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":136,"Executor Deserialize CPU Time":71500541,"Executor Run Time":2182,"Executor CPU Time":1785119941,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349006,"Shuffle Write Time":32035583,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145971,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"5212268","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"9572","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"121634812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"9","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1703","Value":"5517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"525948","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"9573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1609","Value":"4921","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":9573,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":97944567,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":20055909,"Value":79410400,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":9572,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":941007,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":121634816,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":7182,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":896878991,"Value":4180973307,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1722,"Value":5826,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2787355,"Value":123835301,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":195,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2787355,"Executor Run Time":1722,"Executor CPU Time":896878991,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":242714,"Shuffle Write Time":20055909,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} @@ -46,11 +46,11 @@ {"Event":"SparkListenerExecutorAdded","Timestamp":1524182149826,"Executor ID":"7","Executor Info":{"Host":"node6340.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096","stderr":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"node6340.grid.company.com","Port":5933},"Maximum Memory":956615884,"Timestamp":1524182149983,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182152418,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"48","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"6093","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"5951","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":49790497,"Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":920,"Value":1015,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9619,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5299274511,"Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":7847,"Value":13673,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687811857,"Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1037,"Value":1232,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1037,"Executor Deserialize CPU Time":687811857,"Executor Run Time":7847,"Executor CPU Time":5299274511,"Result Size":2437,"JVM GC Time":920,"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":355051,"Shuffle Write Time":49790497,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":629553808,"JVM Used Nonheap Memory":205304696,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":905801,"Offheap Storage Memory":0,"Onheap Unified Memory":905801,"Offheap Unified Memory":0,"Direct Memory":397602,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":595946552,"JVM Used Nonheap Memory":91208368,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":58468944,"Offheap Storage Memory":0,"Onheap Unified Memory":58468944,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":755008624,"JVM Used Nonheap Memory":100519936,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":47962185,"Offheap Storage Memory":0,"Onheap Unified Memory":47962185,"Offheap Unified Memory":0,"Direct Memory":98230,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":518613056,"JVM Used Nonheap Memory":95657456,"Onheap Execution Memory":37748736,"Offheap Execution Memory":0,"Onheap Storage Memory":63104457,"Offheap Storage Memory":0,"Onheap Unified Memory":100853193,"Offheap Unified Memory":0,"Direct Memory":126261,"Mapped Memory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVM Used Heap Memory":726805712,"JVM Used Nonheap Memory":90709624,"Onheap Execution Memory":0,"Offheap Execution Memory":0,"Onheap Storage Memory":69535048,"Offheap Storage Memory":0,"Onheap Unified Memory":69535048,"Offheap Unified Memory":0,"Direct Memory":87796,"Mapped Memory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":629553808,"JVMOffHeapMemory":205304696,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":397602,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":595946552,"JVMOffHeapMemory":91208368,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":58468944,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":58468944,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":755008624,"JVMOffHeapMemory":100519936,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47962185,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47962185,"OffHeapUnifiedMemory":0,"DirectPoolMemory":98230,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} +{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Completion Time":1524182152419,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":1232,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":13673,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":1015,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9619,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} {"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1524182152447,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index b2d4505eaf87..9fe37510247a 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -213,8 +213,8 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) - val executorUpdates = new ExecutorMetrics(1234567L, 123456L, 543L, 12345L, 1234L, 123L, 12L, - 432L, 321L, 654L, 765L) + val executorUpdates = new ExecutorMetrics(1234567L, Array(123456L, 543L, 12345L, 1234L, 123L, + 12L, 432L, 321L, 654L, 765L)) val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId, executorUpdates)) if (executorShouldReregister) { 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 f6ad8a6f16d8..e2637ab0ab20 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -279,16 +279,20 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit Map( ((0, "1"), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))), + new ExecutorMetrics(-1L, + Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), ((0, "2"), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))), + new ExecutorMetrics(-1L, + Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), ((1, "1"), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))), + new ExecutorMetrics(-1L, + Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), ((1, "2"), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + new ExecutorMetrics(-1L, + Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) // Events to post. val events = Array( @@ -298,36 +302,50 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit createExecutorAddedEvent(2), createStageSubmittedEvent(0), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)), + new ExecutorMetrics(10L, + Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)), + new ExecutorMetrics(10L, + Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)), + new ExecutorMetrics(15L, + Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)), + new ExecutorMetrics(15L, + Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)), + new ExecutorMetrics(20L, + Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)), + new ExecutorMetrics(20L, + Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), createStageSubmittedEvent(1), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)), + new ExecutorMetrics(25L, + Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)), + new ExecutorMetrics(25L, + Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), createStageCompletedEvent(0), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)), + new ExecutorMetrics(30L, + Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)), + new ExecutorMetrics(30L, + Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)), + new ExecutorMetrics(35L, + Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)), + new ExecutorMetrics(35L, + Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)), + new ExecutorMetrics(40L, + Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), createExecutorRemovedEvent(1), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)), + new ExecutorMetrics(40L, + Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -350,21 +368,21 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(lines(0).contains("SparkListenerLogStart")) assert(lines(1).contains("SparkListenerApplicationStart")) assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) - var i = 1 + var logIdx = 1 events.foreach {event => event match { case metricsUpdate: SparkListenerExecutorMetricsUpdate => case stageCompleted: SparkListenerStageCompleted => - for (j <- 1 to 2) { - checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId, + (1 to 2).foreach { _ => + checkExecutorMetricsUpdate(lines(logIdx), stageCompleted.stageInfo.stageId, expectedMetricsEvents) - i += 1 - } - checkEvent(lines(i), event) - i += 1 + logIdx += 1 + } + checkEvent(lines(logIdx), event) + logIdx += 1 case _ => - checkEvent(lines(i), event) - i += 1 + checkEvent(lines(logIdx), event) + logIdx += 1 } } } finally { @@ -372,29 +390,24 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } - /** Create a stage submitted event for the specified stage Id. */ private def createStageSubmittedEvent(stageId: Int) = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "details")) } - /** Create a stage completed event for the specified stage Id. */ private def createStageCompletedEvent(stageId: Int) = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "details")) } - /** 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)) } - /** Create an executor added event for the specified executor Id. */ private def createExecutorRemovedEvent(executorId: Int) = { SparkListenerExecutorRemoved(0L, executorId.toString, "test") } - /** Create an executor metrics update event, with the specified executor metrics values. */ private def createExecutorMetricsUpdateEvent( executorId: Int, executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { @@ -424,34 +437,17 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit /** Check that the Spark history log line matches the expected event. */ private def checkEvent(line: String, event: SparkListenerEvent): Unit = { assert(line.contains(event.getClass.toString.split("\\.").last)) - event match { - case executorMetrics: SparkListenerExecutorMetricsUpdate => - JsonProtocol.sparkEventFromJson(parse(line)) match { - case executorMetrics2: SparkListenerExecutorMetricsUpdate => - assert(executorMetrics.execId === executorMetrics2.execId) - assert(executorMetrics2.accumUpdates.isEmpty) - checkExecutorMetrics(executorMetrics.executorUpdates, executorMetrics2.executorUpdates) - case _ => - assertTypeError("expecting SparkListenerExecutorMetricsUpdate") - } - case stageSubmitted: SparkListenerStageSubmitted => + val parsed = JsonProtocol.sparkEventFromJson(parse(line)) + assert(parsed.getClass === event.getClass) + (event, parsed) match { + case (expected: SparkListenerStageSubmitted, actual: SparkListenerStageSubmitted) => // accumulables can be different, so only check the stage Id - JsonProtocol.sparkEventFromJson(parse(line)) match { - case logStageSubmitted : SparkListenerStageSubmitted => - assert(logStageSubmitted.stageInfo.stageId == stageSubmitted.stageInfo.stageId) - case _ => - assertTypeError("expecting SparkListenerStageSubmitted") - } - case stageCompleted: SparkListenerStageCompleted => + assert(expected.stageInfo.stageId == actual.stageInfo.stageId) + case (expected: SparkListenerStageCompleted, actual: SparkListenerStageCompleted) => // accumulables can be different, so only check the stage Id - JsonProtocol.sparkEventFromJson(parse(line)) match { - case logStageSubmitted : SparkListenerStageSubmitted => - assert(logStageSubmitted.stageInfo.stageId == stageCompleted.stageInfo.stageId) - case _ => - assertTypeError("expecting SparkListenerStageCompleted") - } - case _ => - assert(JsonProtocol.sparkEventFromJson(parse(line)) === event) + assert(expected.stageInfo.stageId == actual.stageInfo.stageId) + case (expected: SparkListenerEvent, actual: SparkListenerEvent) => + assert(expected === actual) } } @@ -477,7 +473,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(false) } case _ => - assertTypeError("expecting SparkListenerExecutorMetricsUpdate") + fail("expecting SparkListenerExecutorMetricsUpdate") } } 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 136f20f8db04..a32e77305f5b 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -29,10 +29,10 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.metrics.MetricGetter import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.api.v1 -import org.apache.spark.status.api.v1.PeakMemoryMetrics import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -1218,42 +1218,42 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onExecutorAdded(createExecutorAddedEvent(2)) listener.onStageSubmitted(createStageSubmittedEvent(0)) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + new ExecutorMetrics(10L, Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + new ExecutorMetrics(10L, Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + new ExecutorMetrics(15L, Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + new ExecutorMetrics(15L, Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + new ExecutorMetrics(20L, Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + new ExecutorMetrics(20L, Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)))) listener.onStageSubmitted(createStageSubmittedEvent(1)) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + new ExecutorMetrics(25L, Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(25L, 7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + new ExecutorMetrics(25L, Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)))) listener.onStageCompleted(createStageCompletedEvent(0)) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + new ExecutorMetrics(30L, Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + new ExecutorMetrics(30L, Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + new ExecutorMetrics(35L, Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + new ExecutorMetrics(35L, Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + new ExecutorMetrics(40L, Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)))) listener.onExecutorRemoved(createExecutorRemovedEvent(1)) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + new ExecutorMetrics(40L, Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new PeakMemoryMetrics(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), - "2" -> new PeakMemoryMetrics(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) + "1" -> Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), + "2" -> Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) // check that the stored peak values match the expected values for ((id, metrics) <- expectedValues) { @@ -1261,16 +1261,9 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - assert(actual.jvmUsedHeapMemory == metrics.jvmUsedHeapMemory) - assert(actual.jvmUsedNonHeapMemory == metrics.jvmUsedNonHeapMemory) - assert(actual.onHeapExecutionMemory == metrics.onHeapExecutionMemory) - assert(actual.offHeapExecutionMemory == metrics.offHeapExecutionMemory) - assert(actual.onHeapStorageMemory == metrics.onHeapStorageMemory) - assert(actual.offHeapStorageMemory == metrics.offHeapStorageMemory) - assert(actual.onHeapUnifiedMemory == metrics.onHeapUnifiedMemory) - assert(actual.offHeapUnifiedMemory == metrics.offHeapUnifiedMemory) - assert(actual.directMemory == metrics.directMemory) - assert(actual.mappedMemory == metrics.mappedMemory) + (0 until MetricGetter.values.length).foreach { idx => + assert(actual(idx) === metrics(idx)) + } case _ => assert(false) } 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 4f3b412dbd43..8b720117a6aa 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -95,8 +95,8 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - val executorUpdates = Some(new ExecutorMetrics(1234567L, 543L, 123456L, 12345L, 1234L, 123L, - 12L, 432L, 321L, 654L, 765L)) + val executorUpdates = Some(new ExecutorMetrics(1234567L, Array(543L, 123456L, 12345L, + 1234L, 123L, 12L, 432L, 321L, 654L, 765L))) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) } val blockUpdated = @@ -876,7 +876,7 @@ private[spark] object JsonProtocolSuite extends Assertions { } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(1234567L, 123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L)) + Some(new ExecutorMetrics(1234567L, Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L))) } else { None } @@ -2073,16 +2073,16 @@ private[spark] object JsonProtocolSuite extends Assertions { | ], | "Executor Metrics Updated" : { | "Timestamp" : 1234567, - | "JVM Used Heap Memory" : 543, - | "JVM Used Nonheap Memory" : 123456, - | "Onheap Execution Memory" : 12345, - | "Offheap Execution Memory" : 1234, - | "Onheap Storage Memory" : 123, - | "Offheap Storage Memory" : 12, - | "Onheap Unified Memory" : 432, - | "Offheap Unified Memory" : 321, - | "Direct Memory" : 654, - | "Mapped Memory" : 765 + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765 | } | |} From 263c8c846265b6bdfdce471e44c163ab85b930a3 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Thu, 14 Jun 2018 16:52:11 -0700 Subject: [PATCH 17/29] code review comments --- .../scala/org/apache/spark/SparkContext.scala | 5 --- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../scheduler/EventLoggingListener.scala | 5 ++- .../org/apache/spark/status/api/v1/api.scala | 16 ++++------ .../scheduler/EventLoggingListenerSuite.scala | 32 ++++++++++++++++--- 5 files changed, 38 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5156e11a01ef..c9bf40ed6cbf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -305,11 +305,6 @@ class SparkContext(config: SparkConf) extends Logging { _dagScheduler = ds } - private[spark] def heartbeater: Heartbeater = _heartbeater - private[spark] def heartbeater_=(hb: Heartbeater): Unit = { - _heartbeater = hb - } - /** * A unique identifier for the Spark application. * Its format depends on the scheduler implementation. diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ee99d7aaf056..8d70c22f14e5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1753,7 +1753,7 @@ class DAGScheduler( messageScheduler.shutdownNow() eventProcessLoop.stop() taskScheduler.stop() - } + } eventProcessLoop.start() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index f0da348a7ec0..e431cb21924f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -96,7 +96,7 @@ private[spark] class EventLoggingListener( // Visible for tests only. private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) - // map of live stages, to peak executor metrics for the stage + // map of (stageId, stageAttempt), to peak executor metrics for the stage private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, PeakExecutorMetrics]]() /** @@ -197,6 +197,9 @@ private[spark] class EventLoggingListener( executorMap.foreach { executorEntry => { for ((executorId, peakExecutorMetrics) <- executorEntry) { + // -1 timestamp indicates that the ExecutorMetricsUpdate event is being read from the + // history log, and contains the peak metrics for the stage whose StageCompleted event + // immediately follows val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.metrics) val executorUpdate = new SparkListenerExecutorMetricsUpdate( executorId, accumUpdates, Some(executorMetrics)) 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 d36588ae40da..5fe7330bda76 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 @@ -23,6 +23,7 @@ import scala.xml.{NodeSeq, Text} 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} import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize} @@ -113,26 +114,21 @@ class MemoryMetrics private[spark]( val totalOffHeapStorageMemory: Long) /** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */ -class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] { +class PeakMemoryMetricsDeserializer private[spark] extends JsonDeserializer[Option[Array[Long]]] { override def deserialize( jsonParser: JsonParser, deserializationContext: DeserializationContext): Option[Array[Long]] = { - val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]]) + val metricsMap = jsonParser.readValueAs[Option[Map[String, Long]]]( + new TypeReference[Option[Map[String, java.lang.Long]]] {}) metricsMap match { case Some(metrics) => - Some(MetricGetter.values.map { m => - metrics.getOrElse (m.name, 0L) match { - case intVal: Int => intVal.toLong - case longVal: Long => longVal - } - }.toArray) + Some(MetricGetter.values.map(m => metrics.getOrElse(m.name, 0L)).toArray) case None => None } } } - /** serializer for peakMemoryMetrics: convert array to map with metric name as key */ -class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] { +class PeakMemoryMetricsSerializer private[spark] extends JsonSerializer[Option[Array[Long]]] { override def serialize( metrics: Option[Array[Long]], jsonGenerator: JsonGenerator, 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 e2637ab0ab20..521c9bf3e340 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -21,7 +21,7 @@ import java.io.{File, FileOutputStream, InputStream, IOException} import scala.collection.immutable.Map import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.Set import scala.io.Source import org.apache.hadoop.fs.Path @@ -301,48 +301,66 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit createExecutorAddedEvent(1), createExecutorAddedEvent(2), createStageSubmittedEvent(0), + // receive 3 metric updates from each executor with just stage 0 running, + // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(10L, Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(10L, Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), + // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(15L, Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(15L, Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), + // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(20L, Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(20L, Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), + // now start stage 1, one more metric update for each executor, and new + // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), + // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(25L, Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(25L, Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), + // complete stage 0, and 3 more updates for each executor with just + // stage 1 running createStageCompletedEvent(0), + // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(30L, Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), + // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(30L, Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), + // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(35L, Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), + // exec 2: new stage 1 peak for metrics at index: 7 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(35L, Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), + // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(40L, Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), createExecutorRemovedEvent(1), + // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(40L, Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), @@ -373,11 +391,14 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit event match { case metricsUpdate: SparkListenerExecutorMetricsUpdate => case stageCompleted: SparkListenerStageCompleted => + val execIds = Set[String]() (1 to 2).foreach { _ => - checkExecutorMetricsUpdate(lines(logIdx), stageCompleted.stageInfo.stageId, - expectedMetricsEvents) + val execId = checkExecutorMetricsUpdate(lines(logIdx), + stageCompleted.stageInfo.stageId, expectedMetricsEvents) + execIds += execId logIdx += 1 } + assert(execIds.size == 2) // check that each executor was logged checkEvent(lines(logIdx), event) logIdx += 1 case _ => @@ -462,16 +483,17 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit private def checkExecutorMetricsUpdate( line: String, stageId: Int, - expectedEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate]): Unit = { + expectedEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate]): String = { JsonProtocol.sparkEventFromJson(parse(line)) match { case executorMetrics: SparkListenerExecutorMetricsUpdate => expectedEvents.get((stageId, executorMetrics.execId)) match { case Some(expectedMetrics) => assert(executorMetrics.accumUpdates.isEmpty) checkExecutorMetrics(executorMetrics.executorUpdates, expectedMetrics.executorUpdates) - case None => + case None => assert(false) } + executorMetrics.execId case _ => fail("expecting SparkListenerExecutorMetricsUpdate") } From 812fdcf3961bae2a4fa20b4f60e739b45233fcd0 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Fri, 22 Jun 2018 16:53:23 -0700 Subject: [PATCH 18/29] code review comments: - remove timestamp - change ExecutorMetrics to Array[Long] - create new SparkListenerStageExecutorMetrics for recording stage executor metric peaks in the history log Fix issue where metrics for a removed executor were ignored (save dead executors while there currently active stages that the executor was alive for). --- .../apache/spark/SparkFirehoseListener.java | 6 + .../org/apache/spark/HeartbeatReceiver.scala | 3 +- .../scala/org/apache/spark/Heartbeater.scala | 11 +- .../spark/executor/ExecutorMetrics.scala | 43 ------- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../scheduler/EventLoggingListener.scala | 11 +- .../spark/scheduler/PeakExecutorMetrics.scala | 5 +- .../spark/scheduler/SparkListener.scala | 28 ++++- .../spark/scheduler/SparkListenerBus.scala | 2 + .../spark/scheduler/TaskScheduler.scala | 3 +- .../spark/scheduler/TaskSchedulerImpl.scala | 3 +- .../spark/status/AppStatusListener.scala | 36 +++++- .../org/apache/spark/util/JsonProtocol.scala | 49 ++++---- .../application_1506645932520_24630151 | 20 ++-- .../apache/spark/HeartbeatReceiverSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 5 +- .../scheduler/EventLoggingListenerSuite.scala | 107 +++++++----------- .../ExternalClusterManagerSuite.scala | 3 +- .../spark/status/AppStatusListenerSuite.scala | 96 +++++++++++++--- .../apache/spark/util/JsonProtocolSuite.scala | 84 ++++++++++---- 20 files changed, 310 insertions(+), 215 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 94c5c11b61a5..804d775b77cc 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -103,6 +103,12 @@ public final void onExecutorMetricsUpdate( onEvent(executorMetricsUpdate); } + @Override + public final void onStageExecutorMetrics( + SparkListenerStageExecutorMetrics executorMetrics) { + onEvent(executorMetrics); + } + @Override public final void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { onEvent(executorAdded); diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index ac6fb1f2659c..89f4d0af5b55 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -22,7 +22,6 @@ import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable import scala.concurrent.Future -import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ @@ -39,7 +38,7 @@ private[spark] case class Heartbeat( executorId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates blockManagerId: BlockManagerId, - executorUpdates: ExecutorMetrics) // executor level updates + executorUpdates: Array[Long]) // executor level updates /** * An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index ca644c1a2b36..af050f772c0a 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -19,7 +19,6 @@ package org.apache.spark import java.util.concurrent.TimeUnit -import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.memory.MemoryManager import org.apache.spark.metrics.MetricGetter @@ -59,10 +58,12 @@ private[spark] class Heartbeater( heartbeater.awaitTermination(10, TimeUnit.SECONDS) } - /** Get the current metrics. */ - def getCurrentMetrics(): ExecutorMetrics = { - new ExecutorMetrics(System.currentTimeMillis(), - MetricGetter.values.map(_.getMetricValue(memoryManager)).toArray) + /** + * Get the current executor level metrics. These are returned as an array, with the index + * determined by MetricGetter.values + */ + def getCurrentMetrics(): Array[Long] = { + MetricGetter.values.map(_.getMetricValue(memoryManager)).toArray } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala deleted file mode 100644 index 9305249ccb34..000000000000 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ /dev/null @@ -1,43 +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.executor - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.metrics.MetricGetter - -/** - * :: DeveloperApi :: - * Executor level metrics. - * - * This is sent to the driver periodically (on executor heartbeat), to provide - * information about each executor's metrics. - * - * @param timestamp the time the metrics were collected, or -1 for Spark history - * log events which are logged when a stage has completed - * @param metrics the array of executor metrics values, order and elements as - * specified in MetricGetter - */ -@DeveloperApi -class ExecutorMetrics private[spark] ( - val timestamp: Long, - val metrics: Array[Long]) extends Serializable { - if (metrics.length != MetricGetter.values.length) { - throw new IllegalArgumentException("invalid metrics length " + metrics.length + - " does not equal expected length " + MetricGetter.values.length) - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8d70c22f14e5..0e6c84da805e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -34,7 +34,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils @@ -247,7 +247,7 @@ class DAGScheduler( // (taskId, stageId, stageAttemptId, accumUpdates) accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId, - executorUpdates: ExecutorMetrics): Boolean = { + executorUpdates: Array[Long]): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, Some(executorUpdates))) blockManagerMaster.driverEndpoint.askSync[Boolean]( diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index e431cb21924f..b354e4860326 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -36,7 +36,6 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.io.CompressionCodec @@ -191,19 +190,13 @@ private[spark] class EventLoggingListener( // log the peak executor metrics for the stage, for each live executor, // whether or not the executor is running tasks for the stage - val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]() val executorMap = liveStageExecutorMetrics.remove( (event.stageInfo.stageId, event.stageInfo.attemptNumber())) executorMap.foreach { executorEntry => { for ((executorId, peakExecutorMetrics) <- executorEntry) { - // -1 timestamp indicates that the ExecutorMetricsUpdate event is being read from the - // history log, and contains the peak metrics for the stage whose StageCompleted event - // immediately follows - val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.metrics) - val executorUpdate = new SparkListenerExecutorMetricsUpdate( - executorId, accumUpdates, Some(executorMetrics)) - logEvent(executorUpdate) + logEvent(new SparkListenerStageExecutorMetrics(executorId, event.stageInfo.stageId, + event.stageInfo.attemptNumber(), peakExecutorMetrics.metrics)) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index b7c279e63b45..598189228eac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.metrics.MetricGetter /** @@ -35,11 +34,11 @@ private[spark] class PeakExecutorMetrics { * @param executorMetrics the executor metrics to compare * @return if there is a new peak value for any metric */ - def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = { + def compareAndUpdate(executorMetrics: Array[Long]): Boolean = { var updated: Boolean = false (0 until MetricGetter.values.length).foreach { metricIdx => - val newVal = executorMetrics.metrics(metricIdx) + val newVal = executorMetrics(metricIdx) if ( newVal > metrics(metricIdx)) { updated = true metrics(metricIdx) = newVal 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 effc0fa53e7c..ad37d1994485 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -26,7 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo import org.apache.spark.{SparkConf, TaskEndReason} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.ui.SparkUI @@ -166,7 +166,23 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends case class SparkListenerExecutorMetricsUpdate( execId: String, accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])], - executorUpdates: Option[ExecutorMetrics] = None) + executorUpdates: Option[Array[Long]] = None) + extends SparkListenerEvent + +/** + * Peak metric values for the executor for the stage, written to the history log at stage + * completion. + * @param execId executor id + * @param stageId stage id + * @param stageAttemptId stage attempt + * @param executorMetrics executor level metrics + */ +@DeveloperApi +case class SparkListenerStageExecutorMetrics( + execId: String, + stageId: Int, + stageAttemptId: Int, + executorMetrics: Array[Long]) extends SparkListenerEvent @DeveloperApi @@ -266,6 +282,11 @@ private[spark] trait SparkListenerInterface { */ def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit + /** + * Called when the driver reads stage executor metrics from the history log. + */ + def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit + /** * Called when the driver registers a new executor. */ @@ -363,6 +384,9 @@ abstract class SparkListener extends SparkListenerInterface { override def onExecutorMetricsUpdate( executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { } + override def onStageExecutorMetrics( + executorMetrics: SparkListenerStageExecutorMetrics): Unit = { } + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { } override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index ff19cc65552e..8f6b7ad30960 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -57,6 +57,8 @@ private[spark] trait SparkListenerBus listener.onApplicationEnd(applicationEnd) case metricsUpdate: SparkListenerExecutorMetricsUpdate => listener.onExecutorMetricsUpdate(metricsUpdate) + case stageExecutorMetrics: SparkListenerStageExecutorMetrics => + listener.onStageExecutorMetrics(stageExecutorMetrics) case executorAdded: SparkListenerExecutorAdded => listener.onExecutorAdded(executorAdded) case executorRemoved: SparkListenerExecutorRemoved => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 037e083c4370..4d80eaadd41a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -77,7 +76,7 @@ private[spark] trait TaskScheduler { execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorUpdates: ExecutorMetrics): Boolean + executorUpdates: Array[Long]): Boolean /** * Get an application ID associated with the job. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index cd9bc5f51e1d..df1c13ebf8ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -28,7 +28,6 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState -import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -444,7 +443,7 @@ private[spark] class TaskSchedulerImpl( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: ExecutorMetrics): Boolean = { + executorMetrics: Array[Long]): Boolean = { // (taskId, stageId, stageAttemptId, accumUpdates) val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { accumUpdates.flatMap { case (id, updates) => 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 6a54a589b4f7..cfde0e229fac 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1 @@ -66,6 +66,7 @@ private[spark] class AppStatusListener( private val liveStages = new ConcurrentHashMap[(Int, Int), LiveStage]() private val liveJobs = new HashMap[Int, LiveJob]() private val liveExecutors = new HashMap[String, LiveExecutor]() + private val deadExecutors = new HashMap[String, LiveExecutor]() private val liveTasks = new HashMap[Long, LiveTask]() private val liveRDDs = new HashMap[Int, LiveRDD]() private val pools = new HashMap[String, SchedulerPool]() @@ -204,6 +205,19 @@ private[spark] class AppStatusListener( update(rdd, now) } } + if (isExecutorActiveForLiveStages(exec)) { + // the executor was running for a currently active stage, so save it for now in + // deadExecutors, and remove when there are no active stages overlapping with the + // executor. + deadExecutors.put(event.executorId, exec) + } + } + } + + /** Was the specified executor active for any currently live stages? */ + private def isExecutorActiveForLiveStages(exec: LiveExecutor): Boolean = { + liveStages.values.asScala.exists { stage => + stage.info.submissionTime.getOrElse(0L) < exec.removeTime.getTime } } @@ -618,6 +632,9 @@ private[spark] class AppStatusListener( } } + // remove any dead executors that were not running for any currently active stages + deadExecutors.retain((execId, exec) => isExecutorActiveForLiveStages(exec)) + appSummary = new AppSummary(appSummary.numCompletedJobs, appSummary.numCompletedStages + 1) kvstore.write(appSummary) } @@ -669,7 +686,7 @@ private[spark] class AppStatusListener( } } } - event.executorUpdates.foreach { updates: ExecutorMetrics => + event.executorUpdates.foreach { updates: Array[Long] => // check if there is a new peak value for any of the executor level memory metrics liveExecutors.get(event.execId).foreach { exec: LiveExecutor => if (exec.peakExecutorMetrics.compareAndUpdate(updates)) { @@ -679,6 +696,21 @@ private[spark] class AppStatusListener( } } + override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = { + val now = System.nanoTime() + + // check if there is a new peak value for any of the executor level memory metrics + liveExecutors.get(executorMetrics.execId) + .orElse(deadExecutors.get(executorMetrics.execId)) match { + case Some(exec) => + if (exec.peakExecutorMetrics.compareAndUpdate(executorMetrics.executorMetrics)) { + maybeUpdate(exec, now) + } + case None => + logWarning("unable to find executor " + executorMetrics.execId) + } + } + override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = { event.blockUpdatedInfo.blockId match { case block: RDDBlockId => updateRDDBlock(event, block) 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 e42fc0d94d03..34da386a4f17 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -99,6 +99,8 @@ private[spark] object JsonProtocol { logStartToJson(logStart) case metricsUpdate: SparkListenerExecutorMetricsUpdate => executorMetricsUpdateToJson(metricsUpdate) + case stageExecutorMetrics: SparkListenerStageExecutorMetrics => + stageExecutorMetricsToJson(stageExecutorMetrics) case blockUpdate: SparkListenerBlockUpdated => blockUpdateToJson(blockUpdate) case _ => parse(mapper.writeValueAsString(event)) @@ -247,7 +249,14 @@ private[spark] object JsonProtocol { ("Accumulator Updates" -> JArray(updates.map(accumulableInfoToJson).toList)) }) ~ ("Executor Metrics Updated" -> executorMetrics) + } + def stageExecutorMetricsToJson(metrics: SparkListenerStageExecutorMetrics): JValue = { + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.stageExecutorMetrics) ~ + ("Executor ID" -> metrics.execId) ~ + ("Stage ID" -> metrics.stageId) ~ + ("Stage Attempt ID" -> metrics.stageAttemptId) ~ + ("Executor Metrics" -> executorMetricsToJson(metrics.executorMetrics)) } def blockUpdateToJson(blockUpdate: SparkListenerBlockUpdated): JValue = { @@ -383,19 +392,12 @@ private[spark] object JsonProtocol { ("Updated Blocks" -> updatedBlocks) } - /** - * Convert ExecutorMetrics to JSON. - * - * @param executorMetrics the executor metrics - * @return the JSON representation - */ - def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { + /** Convert executor metrics (indexed by MetricGetter.values) to JSON. */ + def executorMetricsToJson(executorMetrics: Array[Long]): JValue = { val metrics = MetricGetter.idxAndValues.map { case (idx, metric) => - JField(metric.name, executorMetrics.metrics(idx)) + JField(metric.name, executorMetrics(idx)) } - JObject( - (Seq(JField("Timestamp", executorMetrics.timestamp)) ++ metrics): _* - ) + JObject(metrics: _*) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -550,6 +552,7 @@ private[spark] object JsonProtocol { val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved) val logStart = Utils.getFormattedClassName(SparkListenerLogStart) val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate) + val stageExecutorMetrics = Utils.getFormattedClassName(SparkListenerStageExecutorMetrics) val blockUpdate = Utils.getFormattedClassName(SparkListenerBlockUpdated) } @@ -574,6 +577,7 @@ private[spark] object JsonProtocol { case `executorRemoved` => executorRemovedFromJson(json) case `logStart` => logStartFromJson(json) case `metricsUpdate` => executorMetricsUpdateFromJson(json) + case `stageExecutorMetrics` => stageExecutorMetricsFromJson(json) case `blockUpdate` => blockUpdateFromJson(json) case other => mapper.readValue(compact(render(json)), Utils.classForName(other)) .asInstanceOf[SparkListenerEvent] @@ -604,20 +608,13 @@ private[spark] object JsonProtocol { SparkListenerTaskGettingResult(taskInfo) } - /** - * Extract the ExecutorMetrics from JSON. - * - * @param json the JSON representation of executor metrics - * @return the ExecutorMetrics - */ - def executorMetricsFromJson(json: JValue): ExecutorMetrics = { - val timeStamp = (json \ "Timestamp").extract[Long] - + /** Extract the executor metrics (indexed by MetricGetter.values) from JSON. */ + def executorMetricsFromJson(json: JValue): Array[Long] = { val metrics = MetricGetter.values.map {metric => - val metricVal = (json \ metric.name).extract[Long] + val metricVal = jsonOption(json \ metric.name).map(_.extract[Long]).getOrElse(0L) metricVal} - new ExecutorMetrics(timeStamp, metrics.toArray) + metrics.toArray } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { @@ -733,6 +730,14 @@ private[spark] object JsonProtocol { SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates) } + def stageExecutorMetricsFromJson(json: JValue): SparkListenerStageExecutorMetrics = { + val execId = (json \ "Executor ID").extract[String] + val stageId = (json \ "Stage ID").extract[Int] + val stageAttemptId = (json \ "Stage Attempt ID").extract[Int] + val executorMetrics = executorMetricsFromJson(json \ "Executor Metrics") + SparkListenerStageExecutorMetrics(execId, stageId, stageAttemptId, executorMetrics) + } + def blockUpdateFromJson(json: JValue): SparkListenerBlockUpdated = { val blockUpdatedInfo = blockUpdatedInfoFromJson(json \ "Block Updated Info") SparkListenerBlockUpdated(blockUpdatedInfo) diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 index 8d132d57ef6e..c48ed741c56e 100644 --- a/core/src/test/resources/spark-events/application_1506645932520_24630151 +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -33,11 +33,11 @@ {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144246,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"1920974","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"3562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"38","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1813","Value":"1812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"195602","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"3563","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1558","Value":"1557","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":3563,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":36845111,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":27318908,"Value":27318908,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":3562,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":349287,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":33,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1498974375,"Value":1498974375,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1922,"Value":1922,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":49547405,"Value":49547405,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":56,"Executor Deserialize CPU Time":49547405,"Executor Run Time":1922,"Executor CPU Time":1498974375,"Result Size":2394,"JVM GC Time":33,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":27318908,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144444,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"23365","Value":"63634","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":105575962,"Value":301246724,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":326,"Value":1107,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11931694025,"Value":31991331624,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13454,"Value":36578,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":531799977,"Value":1633060096,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":594,"Value":1930,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":594,"Executor Deserialize CPU Time":531799977,"Executor Run Time":13454,"Executor CPU Time":11931694025,"Result Size":1856,"JVM GC Time":326,"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":22950296,"Shuffle Write Time":105575962,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144840,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"24254","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":106963069,"Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":342,"Value":1449,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":12267596062,"Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13858,"Value":50436,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":519573839,"Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":573,"Value":2503,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":573,"Executor Deserialize CPU Time":519573839,"Executor Run Time":13858,"Executor CPU Time":12267596062,"Result Size":1856,"JVM GC Time":342,"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":31362123,"Shuffle Write Time":106963069,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":592412824,"JVMOffHeapMemory":202907152,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":355389,"MappedPoolMemory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":523121272,"JVMOffHeapMemory":88280720,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":52050147,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":52050147,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":214174608,"JVMOffHeapMemory":91548704,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47399168,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47399168,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":592412824,"JVMOffHeapMemory":202907152,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":355389,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":523121272,"JVMOffHeapMemory":88280720,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":52050147,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":52050147,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":214174608,"JVMOffHeapMemory":91548704,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47399168,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47399168,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Completion Time":1524182144852,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":2503,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":50436,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":1449,"Internal":true,"Count Failed Values":true}]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145327,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"49","Value":"86","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"2002","Value":"3814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1755","Value":"3312","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":32035583,"Value":59354491,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4788,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1785119941,"Value":3284094316,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":2182,"Value":4104,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":71500541,"Value":121047946,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":136,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":136,"Executor Deserialize CPU Time":71500541,"Executor Run Time":2182,"Executor CPU Time":1785119941,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349006,"Shuffle Write Time":32035583,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145971,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"5212268","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"9572","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"121634812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"9","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1703","Value":"5517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"525948","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"9573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1609","Value":"4921","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":9573,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":97944567,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":20055909,"Value":79410400,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":9572,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":941007,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":121634816,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":7182,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":896878991,"Value":4180973307,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1722,"Value":5826,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2787355,"Value":123835301,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":195,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2787355,"Executor Run Time":1722,"Executor CPU Time":896878991,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":242714,"Shuffle Write Time":20055909,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} @@ -46,11 +46,11 @@ {"Event":"SparkListenerExecutorAdded","Timestamp":1524182149826,"Executor ID":"7","Executor Info":{"Host":"node6340.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096","stderr":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"node6340.grid.company.com","Port":5933},"Maximum Memory":956615884,"Timestamp":1524182149983,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182152418,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"48","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"6093","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"5951","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":49790497,"Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":920,"Value":1015,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9619,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5299274511,"Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":7847,"Value":13673,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687811857,"Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1037,"Value":1232,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1037,"Executor Deserialize CPU Time":687811857,"Executor Run Time":7847,"Executor CPU Time":5299274511,"Result Size":2437,"JVM GC Time":920,"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":355051,"Shuffle Write Time":49790497,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"driver","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":629553808,"JVMOffHeapMemory":205304696,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":397602,"MappedPoolMemory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"2","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":595946552,"JVMOffHeapMemory":91208368,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":58468944,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":58468944,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"1","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":755008624,"JVMOffHeapMemory":100519936,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47962185,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47962185,"OffHeapUnifiedMemory":0,"DirectPoolMemory":98230,"MappedPoolMemory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"4","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} -{"Event":"SparkListenerExecutorMetricsUpdate","Executor ID":"3","Metrics Updated":[],"Executor Metrics Updated":{"Timestamp":-1,"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":629553808,"JVMOffHeapMemory":205304696,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":397602,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":595946552,"JVMOffHeapMemory":91208368,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":58468944,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":58468944,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":755008624,"JVMOffHeapMemory":100519936,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47962185,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47962185,"OffHeapUnifiedMemory":0,"DirectPoolMemory":98230,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Completion Time":1524182152419,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":1232,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":13673,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":1015,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9619,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} {"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1524182152447,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 9fe37510247a..341ab2e8975a 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -29,7 +29,7 @@ import org.mockito.Matchers._ import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -213,8 +213,8 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) - val executorUpdates = new ExecutorMetrics(1234567L, Array(123456L, 543L, 12345L, 1234L, 123L, - 12L, 432L, 321L, 654L, 765L)) + val executorUpdates = Array(123456L, 543L, 12345L, 1234L, 123L, + 12L, 432L, 321L, 654L, 765L) val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId, executorUpdates)) if (executorShouldReregister) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index db5ad7f30a04..30fca1cb3fee 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -30,7 +30,6 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager -import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} @@ -122,7 +121,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorUpdates: ExecutorMetrics): Boolean = true + executorUpdates: Array[Long]): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -637,7 +636,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: ExecutorMetrics): Boolean = true + executorMetrics: Array[Long]): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None 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 521c9bf3e340..06b9e0dc7b01 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.metrics.{MetricGetter, MetricsSystem} @@ -275,24 +275,20 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val listenerBus = new LiveListenerBus(conf) // expected ExecutorMetricsUpdate, for the given stage id and executor id - val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] = + val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] = Map( ((0, "1"), - createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(-1L, - Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), + new SparkListenerStageExecutorMetrics("1", 0, 0, + Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))), ((0, "2"), - createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(-1L, - Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), + new SparkListenerStageExecutorMetrics("2", 0, 0, + Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))), ((1, "1"), - createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(-1L, - Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), + new SparkListenerStageExecutorMetrics("1", 1, 0, + Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))), ((1, "2"), - createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(-1L, - Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) + new SparkListenerStageExecutorMetrics("2", 1, 0, + Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) // Events to post. val events = Array( @@ -304,66 +300,52 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(10L, - Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), + Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(10L, - Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), + Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)), // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(15L, - Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), + Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)), // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(15L, - Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), + Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)), // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(20L, - Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), + Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(20L, - Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), + Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(25L, - Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), + Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)), // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(25L, - Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), + Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(30L, - Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), + Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)), // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(30L, - Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), + Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)), // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(35L, - Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), + Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)), // exec 2: new stage 1 peak for metrics at index: 7 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(35L, - Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), + Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(40L, - Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), + Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(40L, - Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), + Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -431,7 +413,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit private def createExecutorMetricsUpdateEvent( executorId: Int, - executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { + executorMetrics: Array[Long]): SparkListenerExecutorMetricsUpdate = { val taskMetrics = TaskMetrics.empty taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) @@ -439,22 +421,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) } - /** Check that the two ExecutorMetrics match */ - private def checkExecutorMetrics( - executorMetrics1: Option[ExecutorMetrics], - executorMetrics2: Option[ExecutorMetrics]) = { - (executorMetrics1, executorMetrics2) match { - case (Some(e1), Some(e2)) => - assert(e1.timestamp === e2.timestamp) - (0 until MetricGetter.values.length).foreach { idx => - assert(e1.metrics(idx) === e2.metrics(idx)) - } - case (None, None) => - case _ => - assert(false) - } - } - /** Check that the Spark history log line matches the expected event. */ private def checkEvent(line: String, event: SparkListenerEvent): Unit = { assert(line.contains(event.getClass.toString.split("\\.").last)) @@ -473,8 +439,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } /** - * Check that the Spark history log line is an ExecutorMetricsUpdate, and matches the expected - * value for the stage and executor. + * Check that the Spark history log line is an StageExecutorMetrics event, and matches the + * expected value for the stage and executor. * * @param line the Spark history log line * @param stageId the stage ID the ExecutorMetricsUpdate is associated with @@ -483,19 +449,26 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit private def checkExecutorMetricsUpdate( line: String, stageId: Int, - expectedEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate]): String = { + expectedEvents: Map[(Int, String), SparkListenerStageExecutorMetrics]): String = { JsonProtocol.sparkEventFromJson(parse(line)) match { - case executorMetrics: SparkListenerExecutorMetricsUpdate => + case executorMetrics: SparkListenerStageExecutorMetrics => expectedEvents.get((stageId, executorMetrics.execId)) match { case Some(expectedMetrics) => - assert(executorMetrics.accumUpdates.isEmpty) - checkExecutorMetrics(executorMetrics.executorUpdates, expectedMetrics.executorUpdates) - case None => + assert(executorMetrics.execId === expectedMetrics.execId) + assert(executorMetrics.stageId === expectedMetrics.stageId) + assert(executorMetrics.stageAttemptId === expectedMetrics.stageAttemptId) + assert(executorMetrics.executorMetrics.length === + expectedMetrics.executorMetrics.length) + (0 until MetricGetter.values.length).foreach { idx => + assert(executorMetrics.executorMetrics(idx) === + expectedMetrics.executorMetrics(idx)) + } + case None => assert(false) } executorMetrics.execId case _ => - fail("expecting SparkListenerExecutorMetricsUpdate") + fail("expecting SparkListenerStageExecutorMetrics") } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 149e58fc5a90..4ffd3430b50a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -91,5 +90,5 @@ private class DummyTaskScheduler extends TaskScheduler { execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: ExecutorMetrics): Boolean = true + executorMetrics: Array[Long]): Boolean = true } 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 a32e77305f5b..6eb772a9bdef 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -28,7 +28,7 @@ import scala.reflect.{classTag, ClassTag} import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.metrics.MetricGetter import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ @@ -1217,37 +1217,99 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onExecutorAdded(createExecutorAddedEvent(1)) listener.onExecutorAdded(createExecutorAddedEvent(2)) listener.onStageSubmitted(createStageSubmittedEvent(0)) + // receive 3 metric updates from each executor with just stage 0 running, + // with different peak updates for each executor listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(10L, Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)))) + Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(10L, Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)))) + Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(15L, Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)))) + Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(15L, Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)))) + Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(20L, Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)))) + Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(20L, Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)))) + Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + // now start stage 1, one more metric update for each executor, and new + // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks listener.onStageSubmitted(createStageSubmittedEvent(1)) + // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(25L, Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)))) + Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(25L, Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)))) + Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + // complete stage 0, and 3 more updates for each executor with just + // stage 1 running listener.onStageCompleted(createStageCompletedEvent(0)) + // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(30L, Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)))) + Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(30L, Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)))) + Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(35L, Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)))) + Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + // exec 2: new stage 1 peak for metrics at index: 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(35L, Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)))) + Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + // exec 1: no new stage 1 peaks listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(40L, Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)))) + Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) listener.onExecutorRemoved(createExecutorRemovedEvent(1)) + // exec 2: new stage 1 peak for metrics at index: 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(40L, Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)))) + Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + listener.onStageCompleted(createStageCompletedEvent(1)) + + // expected peak values for each executor + val expectedValues = Map( + "1" -> Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), + "2" -> Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) + + // check that the stored peak values match the expected values + for ((id, metrics) <- expectedValues) { + check[ExecutorSummaryWrapper](id) { exec => + assert(exec.info.id === id) + exec.info.peakMemoryMetrics match { + case Some(actual) => + (0 until MetricGetter.values.length).foreach { idx => + assert(actual(idx) === metrics(idx)) + } + case _ => + assert(false) + } + } + } + } + + test("stage executor metrics") { + // simulate reading in StageExecutorMetrics events from the history log + val listener = new AppStatusListener(store, conf, true) + val driver = BlockManagerId(SparkContext.DRIVER_IDENTIFIER, "localhost", 42) + + listener.onExecutorAdded(createExecutorAddedEvent(1)) + listener.onExecutorAdded(createExecutorAddedEvent(2)) + listener.onStageSubmitted(createStageSubmittedEvent(0)) + listener.onStageSubmitted(createStageSubmittedEvent(1)) + listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 0, 0, + Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))) + listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 0, 0, + Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))) + listener.onStageCompleted(createStageCompletedEvent(0)) + // executor 1 is removed before stage 1 has finished, the stage executor metrics + // are logged afterwards and should still be used to update the executor metrics. + listener.onExecutorRemoved(createExecutorRemovedEvent(1)) + listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 1, 0, + Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))) + listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 1, 0, + Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor @@ -1327,13 +1389,13 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { /** Create an executor added event for the specified executor Id. */ private def createExecutorRemovedEvent(executorId: Int) = { - SparkListenerExecutorRemoved(0L, executorId.toString, "test") + SparkListenerExecutorRemoved(10L, executorId.toString, "test") } /** Create an executor metrics update event, with the specified executor metrics values. */ private def createExecutorMetricsUpdateEvent( executorId: Int, - executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { + executorMetrics: Array[Long]): SparkListenerExecutorMetricsUpdate = { val taskMetrics = TaskMetrics.empty taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) 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 8b720117a6aa..57310623a41d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -95,13 +95,16 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - val executorUpdates = Some(new ExecutorMetrics(1234567L, Array(543L, 123456L, 12345L, - 1234L, 123L, 12L, 432L, 321L, 654L, 765L))) + val executorUpdates = Some(Array(543L, 123456L, 12345L, + 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) } val blockUpdated = SparkListenerBlockUpdated(BlockUpdatedInfo(BlockManagerId("Stars", "In your multitude...", 300), RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100L, 0L)) + val stageExecutorMetrics = + SparkListenerStageExecutorMetrics("1", 2, 3, + Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -127,7 +130,7 @@ class JsonProtocolSuite extends SparkFunSuite { testEvent(nodeUnblacklisted, nodeUnblacklistedJsonString) testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) testEvent(blockUpdated, blockUpdatedJsonString) - testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) + testEvent(stageExecutorMetrics, stageExecutorMetricsJsonString) } test("Dependent Classes") { @@ -464,6 +467,16 @@ class JsonProtocolSuite extends SparkFunSuite { testAccumValue(Some("anything"), 123, JString("123")) } + test("executorMetricsFromJson backward compatibility: handle missing metrics") { + // any missing metrics should be set to 0 + val executorMetrics = Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 789L) + val oldExecutorMetricsJson = + JsonProtocol.executorMetricsToJson(executorMetrics) + .removeField( _._1 == "MappedPoolMemory") + val exepectedExecutorMetrics = Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 0L) + assertExecutorMetricsEquals(exepectedExecutorMetrics, + JsonProtocol.executorMetricsFromJson(oldExecutorMetricsJson)) + } } @@ -593,7 +606,12 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(stageAttemptId1 === stageAttemptId2) assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) }) - assertEquals(e1.executorUpdates, e2.executorUpdates) + assertExecutorMetricsEquals(e1.executorUpdates, e2.executorUpdates) + case (e1: SparkListenerStageExecutorMetrics, e2: SparkListenerStageExecutorMetrics) => + assert(e1.execId === e2.execId) + assert(e1.stageId === e2.stageId) + assert(e1.stageAttemptId === e2.stageAttemptId) + assertExecutorMetricsEquals(e1.executorMetrics, e2.executorMetrics) case (e1, e2) => assert(e1 === e2) case _ => fail("Events don't match in types!") @@ -683,19 +701,6 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(metrics1.bytesRead === metrics2.bytesRead) } - private def assertEquals(metrics1: Option[ExecutorMetrics], metrics2: Option[ExecutorMetrics]) { - (metrics1, metrics2) match { - case (Some(m1), Some(m2)) => - assert(m1.timestamp === m2.timestamp) - (0 until MetricGetter.values.length).foreach { idx => - assert(m1.metrics(idx) === m2.metrics(idx)) - } - case (None, None) => - case _ => - assert(false) - } - } - private def assertEquals(result1: JobResult, result2: JobResult) { (result1, result2) match { case (JobSucceeded, JobSucceeded) => @@ -807,6 +812,26 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(ste1 === ste2) } + private def assertExecutorMetricsEquals( + metrics1: Option[Array[Long]], + metrics2: Option[Array[Long]]) { + (metrics1, metrics2) match { + case (Some(m1), Some(m2)) => + assertExecutorMetricsEquals(m1, m2) + case (None, None) => + case _ => + assert(false) + } + } + + private def assertExecutorMetricsEquals(metrics1: Array[Long], metrics2: Array[Long]) { + assert(metrics1.length === MetricGetter.values.length) + assert(metrics2.length === MetricGetter.values.length) + (0 until MetricGetter.values.length).foreach { idx => + assert(metrics1(idx) === metrics2(idx)) + } + } + /** ----------------------------------- * | Util methods for constructing events | @@ -876,7 +901,7 @@ private[spark] object JsonProtocolSuite extends Assertions { } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(1234567L, Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L))) + Some(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L)) } else { None } @@ -2072,7 +2097,6 @@ private[spark] object JsonProtocolSuite extends Assertions { | } | ], | "Executor Metrics Updated" : { - | "Timestamp" : 1234567, | "JVMHeapMemory" : 543, | "JVMOffHeapMemory" : 123456, | "OnHeapExecutionMemory" : 12345, @@ -2088,6 +2112,28 @@ private[spark] object JsonProtocolSuite extends Assertions { |} """.stripMargin + private val stageExecutorMetricsJsonString = + """ + |{ + | "Event": "SparkListenerStageExecutorMetrics", + | "Executor ID": "1", + | "Stage ID": 2, + | "Stage Attempt ID": 3, + | "Executor Metrics" : { + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765 + | } + |} + """.stripMargin + private val blockUpdatedJsonString = """ |{ From 7ed42a5d0eb0b93bb9ddecf14d9461c80dfe1ea0 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Thu, 28 Jun 2018 11:41:58 -0700 Subject: [PATCH 19/29] Address code review comments. Also make executorUpdates in SparkListenerExecutorMetricsUpdate not optional. These are no longer logged, and backward compatibility should not be an issue. These events should only be used to send task and executor updates for heartbeats, and executors and driver should be the same Spark version. --- .../scala/org/apache/spark/Heartbeater.scala | 4 +- .../scala/org/apache/spark/SparkContext.scala | 5 +- .../spark/internal/config/package.scala | 4 +- .../apache/spark/metrics/MetricGetter.scala | 36 ++++++----- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../scheduler/EventLoggingListener.scala | 28 ++++----- .../spark/scheduler/PeakExecutorMetrics.scala | 7 +-- .../spark/scheduler/SparkListener.scala | 8 ++- .../spark/status/AppStatusListener.scala | 17 +++--- .../org/apache/spark/status/api/v1/api.scala | 6 +- .../org/apache/spark/util/JsonProtocol.scala | 7 +-- .../scheduler/EventLoggingListenerSuite.scala | 22 +++---- .../spark/status/AppStatusListenerSuite.scala | 8 ++- .../apache/spark/util/JsonProtocolSuite.scala | 61 +------------------ project/MimaExcludes.scala | 14 ++--- 15 files changed, 86 insertions(+), 145 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index af050f772c0a..a1ba99385946 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -42,7 +42,7 @@ private[spark] class Heartbeater( private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor(name) /** Schedules a task to report a heartbeat. */ - private[spark] def start(): Unit = { + def start(): Unit = { // Wait a random interval so the heartbeats don't end up in sync val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] @@ -53,7 +53,7 @@ private[spark] class Heartbeater( } /** Stops the heartbeat thread. */ - private[spark] def stop(): Unit = { + def stop(): Unit = { heartbeater.shutdown() heartbeater.awaitTermination(10, TimeUnit.SECONDS) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c9bf40ed6cbf..bf86b898b837 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1928,7 +1928,7 @@ class SparkContext(config: SparkConf) extends Logging { Utils.tryLogNonFatalError { _eventLogger.foreach(_.stop()) } - if(_heartbeater != null) { + if (_heartbeater != null) { Utils.tryLogNonFatalError { _heartbeater.stop() } @@ -2414,8 +2414,7 @@ class SparkContext(config: SparkConf) extends Logging { private def reportHeartBeat(): Unit = { val driverUpdates = _heartbeater.getCurrentMetrics() val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) - listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, - Some(driverUpdates))) + listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, driverUpdates)) } // In order to prevent multiple SparkContexts from being active at the same time, mark this diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 8c0bdc5b1c09..717876650960 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -69,8 +69,8 @@ package object config { .bytesConf(ByteUnit.KiB) .createWithDefaultString("100k") - private[spark] val EVENT_LOG_EXECUTOR_METRICS_UPDATES = - ConfigBuilder("spark.eventLog.logExecutorMetricsUpdates.enabled") + private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS = + ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled") .booleanConf .createWithDefault(true) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala index 27ef6d94744a..edba1abe2e0a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala @@ -21,18 +21,20 @@ import javax.management.ObjectName import org.apache.spark.memory.MemoryManager -sealed trait MetricGetter { +private[spark] sealed trait MetricGetter { def getMetricValue(memoryManager: MemoryManager): Long val name = getClass().getName().stripSuffix("$").split("""\.""").last } -abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter { +private[spark] abstract class MemoryManagerMetricGetter( + f: MemoryManager => Long) extends MetricGetter { override def getMetricValue(memoryManager: MemoryManager): Long = { f(memoryManager) } } -abstract class MBeanMetricGetter(mBeanName: String) extends MetricGetter { +private[spark]abstract class MBeanMetricGetter(mBeanName: String) + extends MetricGetter { val bean = ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean]) @@ -41,36 +43,42 @@ abstract class MBeanMetricGetter(mBeanName: String) extends MetricGetter { } } -case object JVMHeapMemory extends MetricGetter { +private[spark] case object JVMHeapMemory extends MetricGetter { override def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed() } } -case object JVMOffHeapMemory extends MetricGetter { +private[spark] case object JVMOffHeapMemory extends MetricGetter { override def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed() } } -case object OnHeapExecutionMemory extends MemoryManagerMetricGetter(_.onHeapExecutionMemoryUsed) +private[spark] case object OnHeapExecutionMemory extends MemoryManagerMetricGetter( + _.onHeapExecutionMemoryUsed) -case object OffHeapExecutionMemory extends MemoryManagerMetricGetter(_.offHeapExecutionMemoryUsed) +private[spark] case object OffHeapExecutionMemory extends MemoryManagerMetricGetter( + _.offHeapExecutionMemoryUsed) -case object OnHeapStorageMemory extends MemoryManagerMetricGetter(_.onHeapStorageMemoryUsed) +private[spark] case object OnHeapStorageMemory extends MemoryManagerMetricGetter( + _.onHeapStorageMemoryUsed) -case object OffHeapStorageMemory extends MemoryManagerMetricGetter(_.offHeapStorageMemoryUsed) +private[spark] case object OffHeapStorageMemory extends MemoryManagerMetricGetter( + _.offHeapStorageMemoryUsed) -case object OnHeapUnifiedMemory extends MemoryManagerMetricGetter( +private[spark] case object OnHeapUnifiedMemory extends MemoryManagerMetricGetter( (m => m.onHeapExecutionMemoryUsed + m.onHeapStorageMemoryUsed)) -case object OffHeapUnifiedMemory extends MemoryManagerMetricGetter( +private[spark] case object OffHeapUnifiedMemory extends MemoryManagerMetricGetter( (m => m.offHeapExecutionMemoryUsed + m.offHeapStorageMemoryUsed)) -case object DirectPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=direct") -case object MappedPoolMemory extends MBeanMetricGetter("java.nio:type=BufferPool,name=mapped") +private[spark] case object DirectPoolMemory extends MBeanMetricGetter( + "java.nio:type=BufferPool,name=direct") +private[spark] case object MappedPoolMemory extends MBeanMetricGetter( + "java.nio:type=BufferPool,name=mapped") -object MetricGetter { +private[spark] object MetricGetter { val values = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0e6c84da805e..ddfcbf1bb869 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -247,9 +247,9 @@ class DAGScheduler( // (taskId, stageId, stageAttemptId, accumUpdates) accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId, + // executor metrics indexed by MetricGetter.values executorUpdates: Array[Long]): Boolean = { - listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, - Some(executorUpdates))) + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, executorUpdates)) blockManagerMaster.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index b354e4860326..db44aa16e062 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -51,7 +51,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} * spark.eventLog.overwrite - Whether to overwrite any existing files. * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams - * spark.eventLog.logExecutorMetricsUpdates.enabled - Whether to log executor metrics updates + * spark.eventLog.logStageExecutorMetrics.enabled - Whether to log stage executor metrics */ private[spark] class EventLoggingListener( appId: String, @@ -70,7 +70,7 @@ private[spark] class EventLoggingListener( private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) - private val shouldLogExecutorMetricsUpdates = sparkConf.get(EVENT_LOG_EXECUTOR_METRICS_UPDATES) + private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) private val testing = sparkConf.get(EVENT_LOG_TESTING) private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) @@ -162,7 +162,7 @@ private[spark] class EventLoggingListener( // Events that do not trigger a flush override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { logEvent(event) - if (shouldLogExecutorMetricsUpdates) { + if (shouldLogStageExecutorMetrics) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), new HashMap[String, PeakExecutorMetrics]()) @@ -181,7 +181,7 @@ private[spark] class EventLoggingListener( // Events that trigger a flush override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { - if (shouldLogExecutorMetricsUpdates) { + if (shouldLogStageExecutorMetrics) { // clear out any previous attempts, that did not have a stage completed event val prevAttemptId = event.stageInfo.attemptNumber() - 1 for (attemptId <- 0 to prevAttemptId) { @@ -190,14 +190,12 @@ private[spark] class EventLoggingListener( // log the peak executor metrics for the stage, for each live executor, // whether or not the executor is running tasks for the stage - val executorMap = liveStageExecutorMetrics.remove( + val executorOpt = liveStageExecutorMetrics.remove( (event.stageInfo.stageId, event.stageInfo.attemptNumber())) - executorMap.foreach { - executorEntry => { - for ((executorId, peakExecutorMetrics) <- executorEntry) { + executorOpt.foreach { execMap => + execMap.foreach { case (executorId, peakExecutorMetrics) => logEvent(new SparkListenerStageExecutorMetrics(executorId, event.stageInfo.stageId, event.stageInfo.attemptNumber(), peakExecutorMetrics.metrics)) - } } } } @@ -269,14 +267,12 @@ private[spark] class EventLoggingListener( } override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - if (shouldLogExecutorMetricsUpdates) { + if (shouldLogStageExecutorMetrics) { // For the active stages, record any new peak values for the memory metrics for the executor - event.executorUpdates.foreach { executorUpdates => - liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => - val peakMetrics = peakExecutorMetrics.getOrElseUpdate( - event.execId, new PeakExecutorMetrics()) - peakMetrics.compareAndUpdate(executorUpdates) - } + liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => + val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + event.execId, new PeakExecutorMetrics()) + peakMetrics.compareAndUpdate(event.executorUpdates) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala index 598189228eac..de2c9316cd1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala @@ -24,6 +24,7 @@ import org.apache.spark.metrics.MetricGetter * values have been recorded yet. */ private[spark] class PeakExecutorMetrics { + // Metrics are indexed by MetricGetter.values val metrics = new Array[Long](MetricGetter.values.length) metrics(0) = -1 @@ -46,10 +47,4 @@ private[spark] class PeakExecutorMetrics { } updated } - - /** Clears/resets the saved peak values. */ - def reset(): Unit = { - (0 until metrics.length).foreach { idx => metrics(idx) = 0} - metrics(0) = -1 - } } 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 ad37d1994485..b4312a1d1b43 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -166,7 +166,7 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends case class SparkListenerExecutorMetricsUpdate( execId: String, accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])], - executorUpdates: Option[Array[Long]] = None) + executorUpdates: Array[Long]) extends SparkListenerEvent /** @@ -175,7 +175,7 @@ case class SparkListenerExecutorMetricsUpdate( * @param execId executor id * @param stageId stage id * @param stageAttemptId stage attempt - * @param executorMetrics executor level metrics + * @param executorMetrics executor level metrics, indexed by MetricGetter.values */ @DeveloperApi case class SparkListenerStageExecutorMetrics( @@ -283,7 +283,9 @@ private[spark] trait SparkListenerInterface { def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit /** - * Called when the driver reads stage executor metrics from the history log. + * Called with the peak memory metrics for a given (executor, stage) combination. Note that this + * is only present when reading from the event log (as in the history server), and is never + * called in a live application. */ def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit 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 cfde0e229fac..99c2f44b627c 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -686,12 +686,13 @@ private[spark] class AppStatusListener( } } } - event.executorUpdates.foreach { updates: Array[Long] => - // check if there is a new peak value for any of the executor level memory metrics - liveExecutors.get(event.execId).foreach { exec: LiveExecutor => - if (exec.peakExecutorMetrics.compareAndUpdate(updates)) { - maybeUpdate(exec, now) - } + + // check if there is a new peak value for any of the executor level memory metrics + // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed + // for the live UI. + liveExecutors.get(event.execId).foreach { exec: LiveExecutor => + if (exec.peakExecutorMetrics.compareAndUpdate(event.executorUpdates)) { + maybeUpdate(exec, now) } } } @@ -699,7 +700,9 @@ private[spark] class AppStatusListener( override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = { val now = System.nanoTime() - // check if there is a new peak value for any of the executor level memory metrics + // check if there is a new peak value for any of the executor level memory metrics, + // while reading from the log. SparkListenerStageExecutorMetrics are only processed + // when reading logs. liveExecutors.get(executorMetrics.execId) .orElse(deadExecutors.get(executorMetrics.execId)) match { case Some(exec) => 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 5fe7330bda76..20c89b054485 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 @@ -113,8 +113,8 @@ class MemoryMetrics private[spark]( val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) -/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */ -class PeakMemoryMetricsDeserializer private[spark] extends JsonDeserializer[Option[Array[Long]]] { +/** deserializer for peakMemoryMetrics: convert to array ordered by metric name */ +private class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] { override def deserialize( jsonParser: JsonParser, deserializationContext: DeserializationContext): Option[Array[Long]] = { @@ -128,7 +128,7 @@ class PeakMemoryMetricsDeserializer private[spark] extends JsonDeserializer[Opti } } /** serializer for peakMemoryMetrics: convert array to map with metric name as key */ -class PeakMemoryMetricsSerializer private[spark] extends JsonSerializer[Option[Array[Long]]] { +private class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] { override def serialize( metrics: Option[Array[Long]], jsonGenerator: JsonGenerator, 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 34da386a4f17..43020888a412 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -239,7 +239,7 @@ private[spark] object JsonProtocol { def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = { val execId = metricsUpdate.execId val accumUpdates = metricsUpdate.accumUpdates - val executorMetrics = metricsUpdate.executorUpdates.map(executorMetricsToJson(_)) + val executorMetrics = executorMetricsToJson(metricsUpdate.executorUpdates) ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.metricsUpdate) ~ ("Executor ID" -> execId) ~ ("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) => @@ -723,10 +723,7 @@ private[spark] object JsonProtocol { (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) (taskId, stageId, stageAttemptId, updates) } - val executorUpdates = jsonOption(json \ "Executor Metrics Updated") match { - case None => None - case Some(executorUpdate) => Some(executorMetricsFromJson(executorUpdate)) - } + val executorUpdates = executorMetricsFromJson(json \ "Executor Metrics Updated") SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates) } 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 06b9e0dc7b01..f0fda97e9a39 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -144,7 +144,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } test("Executor metrics update") { - testExecutorMetricsUpdateEventLogging() + testStageExecutorMetricsEventLogging() } /* ----------------- * @@ -262,19 +262,17 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } /** - * Test executor metrics update logging functionality. This checks that a - * SparkListenerExecutorMetricsUpdate event is added to the Spark history - * log if one of the executor metrics is larger than any previously - * recorded value for the metric, per executor per stage. The task metrics - * should not be added. + * Test stage executor metrics logging functionality. This checks that peak + * values from SparkListenerExecutorMetricsUpdate events during a stage are + * logged in a StageExecutorMetrics event for each executor at stage completion. */ - private def testExecutorMetricsUpdateEventLogging() { + private def testStageExecutorMetricsEventLogging() { val conf = getLoggingConf(testDirPath, None) - val logName = "executorMetricsUpdated-test" + val logName = "stageExecutorMetrics-test" val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus(conf) - // expected ExecutorMetricsUpdate, for the given stage id and executor id + // expected StageExecutorMetrics, for the given stage id and executor id val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] = Map( ((0, "1"), @@ -375,7 +373,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit case stageCompleted: SparkListenerStageCompleted => val execIds = Set[String]() (1 to 2).foreach { _ => - val execId = checkExecutorMetricsUpdate(lines(logIdx), + val execId = checkStageExecutorMetrics(lines(logIdx), stageCompleted.stageInfo.stageId, expectedMetricsEvents) execIds += execId logIdx += 1 @@ -418,7 +416,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) - SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorMetrics) } /** Check that the Spark history log line matches the expected event. */ @@ -446,7 +444,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit * @param stageId the stage ID the ExecutorMetricsUpdate is associated with * @param expectedEvents map of expected ExecutorMetricsUpdate events, for (stageId, executorId) */ - private def checkExecutorMetricsUpdate( + private def checkStageExecutorMetrics( line: String, stageId: Int, expectedEvents: Map[(Int, String), SparkListenerStageExecutorMetrics]): String = { 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 6eb772a9bdef..d94659e4bb47 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -235,9 +235,11 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { s1Tasks.foreach { task => val accum = new AccumulableInfo(1L, Some(InternalAccumulator.MEMORY_BYTES_SPILLED), Some(value), None, true, false, None) + val executorUpdates = Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate( task.executorId, - Seq((task.taskId, stages.head.stageId, stages.head.attemptNumber, Seq(accum))))) + Seq((task.taskId, stages.head.stageId, stages.head.attemptNumber, Seq(accum))), + executorUpdates)) } check[StageDataWrapper](key(stages.head)) { stage => @@ -1274,7 +1276,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { "2" -> Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) // check that the stored peak values match the expected values - for ((id, metrics) <- expectedValues) { + expectedValues.foreach { case (id, metrics) => check[ExecutorSummaryWrapper](id) { exec => assert(exec.info.id === id) exec.info.peakMemoryMetrics match { @@ -1400,6 +1402,6 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) - SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorMetrics) } } 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 57310623a41d..ab4a3874707e 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -95,8 +95,7 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - val executorUpdates = Some(Array(543L, 123456L, 12345L, - 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) + val executorUpdates = Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) } val blockUpdated = @@ -426,30 +425,6 @@ class JsonProtocolSuite extends SparkFunSuite { exceptionFailure.accumUpdates, oldExceptionFailure.accumUpdates, (x, y) => x == y) } - test("ExecutorMetricsUpdate backward compatibility: executor metrics update") { - // executorMetricsUpdate was added in 2.1.0. For older event logs, this should - // be set to None. - val executorMetricsUpdate = makeExecutorMetricsUpdate("1", true, true) - val oldExecutorMetricsUpdateJson = - JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) - .removeField( _._1 == "Executor Metrics Updated") - val exepectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", true, false) - assertEquals(exepectedExecutorMetricsUpdate, - JsonProtocol.executorMetricsUpdateFromJson(oldExecutorMetricsUpdateJson)) - } - - test("ExecutorMetricsUpdate: empty metrics update") { - // For SparkListenerExecutorMetricUpdate events, metrics update will be set to - // empty in the event log, to avoid excess logging. - val executorMetricsUpdate = makeExecutorMetricsUpdate("1", true, true) - val oldExecutorMetricsUpdateJson = - JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) - .replace(List("Metrics Updated"), List.empty[JValue]) - val exepectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", false, true) - assertEquals(exepectedExecutorMetricsUpdate, - JsonProtocol.executorMetricsUpdateFromJson(oldExecutorMetricsUpdateJson)) - } - test("AccumulableInfo value de/serialization") { import InternalAccumulator._ val blocks = Seq[(BlockId, BlockStatus)]( @@ -812,18 +787,6 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(ste1 === ste2) } - private def assertExecutorMetricsEquals( - metrics1: Option[Array[Long]], - metrics2: Option[Array[Long]]) { - (metrics1, metrics2) match { - case (Some(m1), Some(m2)) => - assertExecutorMetricsEquals(m1, m2) - case (None, None) => - case _ => - assert(false) - } - } - private def assertExecutorMetricsEquals(metrics1: Array[Long], metrics2: Array[Long]) { assert(metrics1.length === MetricGetter.values.length) assert(metrics2.length === MetricGetter.values.length) @@ -832,7 +795,6 @@ private[spark] object JsonProtocolSuite extends Assertions { } } - /** ----------------------------------- * | Util methods for constructing events | * ------------------------------------ */ @@ -887,27 +849,6 @@ private[spark] object JsonProtocolSuite extends Assertions { new AccumulableInfo(id, Some(s"Accumulable$id"), Some(s"delta$id"), Some(s"val$id"), internal, countFailedValues, metadata) - /** Creates an SparkListenerExecutorMetricsUpdate event */ - private def makeExecutorMetricsUpdate( - execId: String, - includeTaskMetrics: Boolean, - includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { - val taskMetrics = - if (includeTaskMetrics) { - Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), - makeAccumulableInfo(2, false, false, None)))) - } else { - Seq() - } - val executorMetricsUpdate = - if (includeExecutorMetrics) { - Some(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L)) - } else { - None - } - SparkListenerExecutorMetricsUpdate(execId, taskMetrics, executorMetricsUpdate) - } - /** * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is * set to true) or read data from a shuffle otherwise. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ebca8c85deec..37fcd3326cd2 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,12 @@ object MimaExcludes { // Exclude rules for 2.4.x lazy val v24excludes = v23excludes ++ Seq( + // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate$"), + // [SPARK-20087][CORE] Attach accumulators / metrics to 'TaskKilled' end reason ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.apply"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.TaskKilled.copy"), @@ -89,13 +95,7 @@ object MimaExcludes { ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), - ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), - - // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.this"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate$") + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol") ) // Exclude rules for 2.3.x From 8d9acdf32984c0c9c621a058b45805872bb9e4c5 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Fri, 29 Jun 2018 16:27:51 -0700 Subject: [PATCH 20/29] Revert and make executorUpdates in SparkListenerExecutorMetricsUpdate optional again, in case of existing users of SparkListenerExecutorMetricsUpdate. --- .../scala/org/apache/spark/SparkContext.scala | 3 +- .../apache/spark/scheduler/DAGScheduler.scala | 3 +- .../scheduler/EventLoggingListener.scala | 10 +-- .../spark/scheduler/SparkListener.scala | 2 +- .../spark/status/AppStatusListener.scala | 8 ++- .../org/apache/spark/util/JsonProtocol.scala | 7 +- .../scheduler/EventLoggingListenerSuite.scala | 2 +- .../spark/status/AppStatusListenerSuite.scala | 6 +- .../apache/spark/util/JsonProtocolSuite.scala | 69 +++++++++++++++---- 9 files changed, 81 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index bf86b898b837..b6275d1f3a00 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2414,7 +2414,8 @@ class SparkContext(config: SparkConf) extends Logging { private def reportHeartBeat(): Unit = { val driverUpdates = _heartbeater.getCurrentMetrics() val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) - listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, driverUpdates)) + listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, + Some(driverUpdates))) } // In order to prevent multiple SparkContexts from being active at the same time, mark this diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ddfcbf1bb869..2c8b0c0dd088 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -249,7 +249,8 @@ class DAGScheduler( blockManagerId: BlockManagerId, // executor metrics indexed by MetricGetter.values executorUpdates: Array[Long]): Boolean = { - listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, executorUpdates)) + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, + Some(executorUpdates))) blockManagerMaster.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index db44aa16e062..cecc699ec31b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -269,10 +269,12 @@ private[spark] class EventLoggingListener( override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { if (shouldLogStageExecutorMetrics) { // For the active stages, record any new peak values for the memory metrics for the executor - liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => - val peakMetrics = peakExecutorMetrics.getOrElseUpdate( - event.execId, new PeakExecutorMetrics()) - peakMetrics.compareAndUpdate(event.executorUpdates) + event.executorUpdates.foreach { executorUpdates => + liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => + val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + event.execId, new PeakExecutorMetrics()) + peakMetrics.compareAndUpdate(executorUpdates) + } } } } 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 b4312a1d1b43..ff908ec8fff5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -166,7 +166,7 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends case class SparkListenerExecutorMetricsUpdate( execId: String, accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])], - executorUpdates: Array[Long]) + executorUpdates: Option[Array[Long]] = None) 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 99c2f44b627c..b182877d416e 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -690,9 +690,11 @@ private[spark] class AppStatusListener( // check if there is a new peak value for any of the executor level memory metrics // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed // for the live UI. - liveExecutors.get(event.execId).foreach { exec: LiveExecutor => - if (exec.peakExecutorMetrics.compareAndUpdate(event.executorUpdates)) { - maybeUpdate(exec, now) + event.executorUpdates.foreach { updates: Array[Long] => + liveExecutors.get(event.execId).foreach { exec: LiveExecutor => + if (exec.peakExecutorMetrics.compareAndUpdate(updates)) { + maybeUpdate(exec, now) + } } } } 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 43020888a412..34da386a4f17 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -239,7 +239,7 @@ private[spark] object JsonProtocol { def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = { val execId = metricsUpdate.execId val accumUpdates = metricsUpdate.accumUpdates - val executorMetrics = executorMetricsToJson(metricsUpdate.executorUpdates) + val executorMetrics = metricsUpdate.executorUpdates.map(executorMetricsToJson(_)) ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.metricsUpdate) ~ ("Executor ID" -> execId) ~ ("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) => @@ -723,7 +723,10 @@ private[spark] object JsonProtocol { (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) (taskId, stageId, stageAttemptId, updates) } - val executorUpdates = executorMetricsFromJson(json \ "Executor Metrics Updated") + val executorUpdates = jsonOption(json \ "Executor Metrics Updated") match { + case None => None + case Some(executorUpdate) => Some(executorMetricsFromJson(executorUpdate)) + } SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates) } 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 f0fda97e9a39..4e3dd054ce50 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -416,7 +416,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) - SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorMetrics) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) } /** Check that the Spark history log line matches the expected event. */ 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 d94659e4bb47..47ed6efb5f4e 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -235,11 +235,9 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { s1Tasks.foreach { task => val accum = new AccumulableInfo(1L, Some(InternalAccumulator.MEMORY_BYTES_SPILLED), Some(value), None, true, false, None) - val executorUpdates = Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate( task.executorId, - Seq((task.taskId, stages.head.stageId, stages.head.attemptNumber, Seq(accum))), - executorUpdates)) + Seq((task.taskId, stages.head.stageId, stages.head.attemptNumber, Seq(accum))))) } check[StageDataWrapper](key(stages.head)) { stage => @@ -1402,6 +1400,6 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) - SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorMetrics) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) } } 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 ab4a3874707e..5f5129ad051c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -96,7 +96,8 @@ class JsonProtocolSuite extends SparkFunSuite { .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } val executorUpdates = Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L) - SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), executorUpdates) + SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), + Some(executorUpdates)) } val blockUpdated = SparkListenerBlockUpdated(BlockUpdatedInfo(BlockManagerId("Stars", @@ -425,6 +426,28 @@ class JsonProtocolSuite extends SparkFunSuite { exceptionFailure.accumUpdates, oldExceptionFailure.accumUpdates, (x, y) => x == y) } + test("ExecutorMetricsUpdate backward compatibility: executor metrics update") { + // executorMetricsUpdate was added in 2.4.0. + val executorMetricsUpdate = makeExecutorMetricsUpdate("1", true, true) + val oldExecutorMetricsUpdateJson = + JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) + .removeField( _._1 == "Executor Metrics Updated") + val exepectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", true, false) + assertEquals(exepectedExecutorMetricsUpdate, + JsonProtocol.executorMetricsUpdateFromJson(oldExecutorMetricsUpdateJson)) + } + + test("executorMetricsFromJson backward compatibility: handle missing metrics") { + // any missing metrics should be set to 0 + val executorMetrics = Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 789L) + val oldExecutorMetricsJson = + JsonProtocol.executorMetricsToJson(executorMetrics) + .removeField( _._1 == "MappedPoolMemory") + val exepectedExecutorMetrics = Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 0L) + assertExecutorMetricsEquals(exepectedExecutorMetrics, + JsonProtocol.executorMetricsFromJson(oldExecutorMetricsJson)) + } + test("AccumulableInfo value de/serialization") { import InternalAccumulator._ val blocks = Seq[(BlockId, BlockStatus)]( @@ -441,17 +464,6 @@ class JsonProtocolSuite extends SparkFunSuite { testAccumValue(Some("anything"), blocks, JString(blocks.toString)) testAccumValue(Some("anything"), 123, JString("123")) } - - test("executorMetricsFromJson backward compatibility: handle missing metrics") { - // any missing metrics should be set to 0 - val executorMetrics = Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 789L) - val oldExecutorMetricsJson = - JsonProtocol.executorMetricsToJson(executorMetrics) - .removeField( _._1 == "MappedPoolMemory") - val exepectedExecutorMetrics = Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 0L) - assertExecutorMetricsEquals(exepectedExecutorMetrics, - JsonProtocol.executorMetricsFromJson(oldExecutorMetricsJson)) - } } @@ -787,6 +799,18 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(ste1 === ste2) } + private def assertExecutorMetricsEquals( + metrics1: Option[Array[Long]], + metrics2: Option[Array[Long]]) { + (metrics1, metrics2) match { + case (Some(m1), Some(m2)) => + assertExecutorMetricsEquals(m1, m2) + case (None, None) => + case _ => + assert(false) + } + } + private def assertExecutorMetricsEquals(metrics1: Array[Long], metrics2: Array[Long]) { assert(metrics1.length === MetricGetter.values.length) assert(metrics2.length === MetricGetter.values.length) @@ -849,6 +873,27 @@ private[spark] object JsonProtocolSuite extends Assertions { new AccumulableInfo(id, Some(s"Accumulable$id"), Some(s"delta$id"), Some(s"val$id"), internal, countFailedValues, metadata) + /** Creates an SparkListenerExecutorMetricsUpdate event */ + private def makeExecutorMetricsUpdate( + execId: String, + includeTaskMetrics: Boolean, + includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { + val taskMetrics = + if (includeTaskMetrics) { + Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), + makeAccumulableInfo(2, false, false, None)))) + } else { + Seq() + } + val executorMetricsUpdate = + if (includeExecutorMetrics) { + Some(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L)) + } else { + None + } + SparkListenerExecutorMetricsUpdate(execId, taskMetrics, executorMetricsUpdate) + } + /** * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is * set to true) or read data from a shuffle otherwise. From 20799d2af7b70334534be913f7defea6d6b79ffb Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Wed, 25 Jul 2018 11:02:45 -0700 Subject: [PATCH 21/29] code review comments: hid array implementation of executor metrics, and add ExecutorMetrics, with getMetricValue() method for accessing executor metric values. Rename MetricGetter to ExecutorMetricType. Should ExecutorMetricType be moved to executor package, or ExecutorMetrics be moved to metrics package? Should Json (de)serialization functions be moved from api.scala to ExecutorMetrics? --- .../org/apache/spark/HeartbeatReceiver.scala | 3 +- .../scala/org/apache/spark/Heartbeater.scala | 8 +- .../spark/executor/ExecutorMetrics.scala | 81 +++++++++++++++++++ ...cGetter.scala => ExecutorMetricType.scala} | 56 +++++++------ .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../scheduler/EventLoggingListener.scala | 12 +-- .../spark/scheduler/PeakExecutorMetrics.scala | 50 ------------ .../spark/scheduler/SparkListener.scala | 6 +- .../spark/scheduler/TaskScheduler.scala | 3 +- .../spark/scheduler/TaskSchedulerImpl.scala | 3 +- .../spark/status/AppStatusListener.scala | 8 +- .../org/apache/spark/status/LiveEntity.scala | 8 +- .../org/apache/spark/status/api/v1/api.scala | 31 +++---- .../org/apache/spark/util/JsonProtocol.scala | 24 +++--- .../apache/spark/HeartbeatReceiverSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 5 +- .../scheduler/EventLoggingListenerSuite.scala | 57 +++++++------ .../ExternalClusterManagerSuite.scala | 3 +- .../spark/scheduler/ReplayListenerSuite.scala | 8 +- .../spark/status/AppStatusListenerSuite.scala | 31 +++---- .../apache/spark/util/JsonProtocolSuite.scala | 48 +++++------ 21 files changed, 250 insertions(+), 205 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala rename core/src/main/scala/org/apache/spark/metrics/{MetricGetter.scala => ExecutorMetricType.scala} (51%) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 89f4d0af5b55..ac6fb1f2659c 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable import scala.concurrent.Future +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ @@ -38,7 +39,7 @@ private[spark] case class Heartbeat( executorId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates blockManagerId: BlockManagerId, - executorUpdates: Array[Long]) // executor level updates + executorUpdates: ExecutorMetrics) // executor level updates /** * An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index a1ba99385946..5ba1b9b2d828 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -19,9 +19,10 @@ package org.apache.spark import java.util.concurrent.TimeUnit +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.memory.MemoryManager -import org.apache.spark.metrics.MetricGetter +import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -62,8 +63,9 @@ private[spark] class Heartbeater( * Get the current executor level metrics. These are returned as an array, with the index * determined by MetricGetter.values */ - def getCurrentMetrics(): Array[Long] = { - MetricGetter.values.map(_.getMetricValue(memoryManager)).toArray + def getCurrentMetrics(): ExecutorMetrics = { + val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray + new ExecutorMetrics(metrics) } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala new file mode 100644 index 000000000000..36544cf65ec6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -0,0 +1,81 @@ +/* + * 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.executor + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.metrics.ExecutorMetricType + +/** + * :: DeveloperApi :: + * Metrics tracked for executors and the driver. + * + * Executor-level metrics are sent from each executor to the driver as part of the Heartbeat. + */ +@DeveloperApi +class ExecutorMetrics private[spark] extends Serializable { + + // Metrics are indexed by MetricGetter.values + private val metrics = new Array[Long](ExecutorMetricType.values.length) + + // the first element is initialized to -1, indicating that the values for the array + // haven't been set yet. + metrics(0) = -1 + + /** Returns the value for the specified metricType. */ + def getMetricValue(metricType: ExecutorMetricType): Long = { + metrics(ExecutorMetricType.metricIdxMap(metricType)) + } + + /** Returns true if the values for the metrics have been set, false otherwise. */ + def isSet(): Boolean = metrics(0) > -1 + + private[spark] def this(metrics: Array[Long]) { + this() + Array.copy(metrics, 0, this.metrics, 0, Math.min(metrics.size, this.metrics.size)) + } + + /** + * Constructor: create the ExecutorMetrics with the values specified. + * + * @param executorMetrics map of executor metric name to value + */ + private[spark] def this(executorMetrics: Map[String, Long]) { + this() + (0 until ExecutorMetricType.values.length).foreach { idx => + metrics(idx) = executorMetrics.getOrElse(ExecutorMetricType.values(idx).name, 0L) + } + } + + /** + * Compare the specified executor metrics values with the current executor metric values, + * and update the value for any metrics where the new value for the metric is larger. + * + * @param executorMetrics the executor metrics to compare + * @return if there is a new peak value for any metric + */ + private[spark] def compareAndUpdatePeakValues(executorMetrics: ExecutorMetrics): Boolean = { + var updated: Boolean = false + + (0 until ExecutorMetricType.values.length).foreach { idx => + if ( executorMetrics.metrics(idx) > metrics(idx)) { + updated = true + metrics(idx) = executorMetrics.metrics(idx) + } + } + updated + } +} diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala similarity index 51% rename from core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala rename to core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index edba1abe2e0a..456663e017c2 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -21,64 +21,70 @@ import javax.management.ObjectName import org.apache.spark.memory.MemoryManager -private[spark] sealed trait MetricGetter { - def getMetricValue(memoryManager: MemoryManager): Long - val name = getClass().getName().stripSuffix("$").split("""\.""").last +/** + * Executor metric types for executor-level metrics stored in ExecutorMetrics. + */ +sealed trait ExecutorMetricType { + private[spark] def getMetricValue(memoryManager: MemoryManager): Long + private[spark] val name = getClass().getName().stripSuffix("$").split("""\.""").last } -private[spark] abstract class MemoryManagerMetricGetter( - f: MemoryManager => Long) extends MetricGetter { - override def getMetricValue(memoryManager: MemoryManager): Long = { +private[spark] abstract class MemoryManagerExecutorMetricType( + f: MemoryManager => Long) extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { f(memoryManager) } } -private[spark]abstract class MBeanMetricGetter(mBeanName: String) - extends MetricGetter { - val bean = ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer, +private[spark]abstract class MBeanExecutorMetricType(mBeanName: String) + extends ExecutorMetricType { + private val bean = ManagementFactory.newPlatformMXBeanProxy( + ManagementFactory.getPlatformMBeanServer, new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean]) - override def getMetricValue(memoryManager: MemoryManager): Long = { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { bean.getMemoryUsed } } -private[spark] case object JVMHeapMemory extends MetricGetter { - override def getMetricValue(memoryManager: MemoryManager): Long = { +case object JVMHeapMemory extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed() } } -private[spark] case object JVMOffHeapMemory extends MetricGetter { - override def getMetricValue(memoryManager: MemoryManager): Long = { +case object JVMOffHeapMemory extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed() } } -private[spark] case object OnHeapExecutionMemory extends MemoryManagerMetricGetter( +case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType( _.onHeapExecutionMemoryUsed) -private[spark] case object OffHeapExecutionMemory extends MemoryManagerMetricGetter( +case object OffHeapExecutionMemory extends MemoryManagerExecutorMetricType( _.offHeapExecutionMemoryUsed) -private[spark] case object OnHeapStorageMemory extends MemoryManagerMetricGetter( +case object OnHeapStorageMemory extends MemoryManagerExecutorMetricType( _.onHeapStorageMemoryUsed) -private[spark] case object OffHeapStorageMemory extends MemoryManagerMetricGetter( +case object OffHeapStorageMemory extends MemoryManagerExecutorMetricType( _.offHeapStorageMemoryUsed) -private[spark] case object OnHeapUnifiedMemory extends MemoryManagerMetricGetter( +case object OnHeapUnifiedMemory extends MemoryManagerExecutorMetricType( (m => m.onHeapExecutionMemoryUsed + m.onHeapStorageMemoryUsed)) -private[spark] case object OffHeapUnifiedMemory extends MemoryManagerMetricGetter( +case object OffHeapUnifiedMemory extends MemoryManagerExecutorMetricType( (m => m.offHeapExecutionMemoryUsed + m.offHeapStorageMemoryUsed)) -private[spark] case object DirectPoolMemory extends MBeanMetricGetter( +case object DirectPoolMemory extends MBeanExecutorMetricType( "java.nio:type=BufferPool,name=direct") -private[spark] case object MappedPoolMemory extends MBeanMetricGetter( + +case object MappedPoolMemory extends MBeanExecutorMetricType( "java.nio:type=BufferPool,name=mapped") -private[spark] object MetricGetter { +private[spark] object ExecutorMetricType { + // List of all executor metric types val values = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, @@ -92,5 +98,7 @@ private[spark] object MetricGetter { MappedPoolMemory ) - val idxAndValues = values.zipWithIndex.map(_.swap) + // Map of executor metric type to its index in values. + val metricIdxMap = + Map[ExecutorMetricType, Int](ExecutorMetricType.values.zipWithIndex: _*) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2c8b0c0dd088..5ce69cd9e097 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -34,7 +34,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.network.util.JavaUtils @@ -248,7 +248,7 @@ class DAGScheduler( accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId, // executor metrics indexed by MetricGetter.values - executorUpdates: Array[Long]): Boolean = { + executorUpdates: ExecutorMetrics): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, Some(executorUpdates))) blockManagerMaster.driverEndpoint.askSync[Boolean]( diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index cecc699ec31b..dd33505fc687 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -23,7 +23,6 @@ import java.nio.charset.StandardCharsets import java.util.EnumSet import java.util.Locale -import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.hadoop.conf.Configuration @@ -36,6 +35,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.io.CompressionCodec @@ -96,7 +96,7 @@ private[spark] class EventLoggingListener( private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) // map of (stageId, stageAttempt), to peak executor metrics for the stage - private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, PeakExecutorMetrics]]() + private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, ExecutorMetrics]]() /** * Creates the log file in the configured log directory. @@ -165,7 +165,7 @@ private[spark] class EventLoggingListener( if (shouldLogStageExecutorMetrics) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), - new HashMap[String, PeakExecutorMetrics]()) + new HashMap[String, ExecutorMetrics]()) } } @@ -195,7 +195,7 @@ private[spark] class EventLoggingListener( executorOpt.foreach { execMap => execMap.foreach { case (executorId, peakExecutorMetrics) => logEvent(new SparkListenerStageExecutorMetrics(executorId, event.stageInfo.stageId, - event.stageInfo.attemptNumber(), peakExecutorMetrics.metrics)) + event.stageInfo.attemptNumber(), peakExecutorMetrics)) } } } @@ -272,8 +272,8 @@ private[spark] class EventLoggingListener( event.executorUpdates.foreach { executorUpdates => liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => val peakMetrics = peakExecutorMetrics.getOrElseUpdate( - event.execId, new PeakExecutorMetrics()) - peakMetrics.compareAndUpdate(executorUpdates) + event.execId, new ExecutorMetrics()) + peakMetrics.compareAndUpdatePeakValues(executorUpdates) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala deleted file mode 100644 index de2c9316cd1b..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala +++ /dev/null @@ -1,50 +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.scheduler - -import org.apache.spark.metrics.MetricGetter - -/** - * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no - * values have been recorded yet. - */ -private[spark] class PeakExecutorMetrics { - // Metrics are indexed by MetricGetter.values - val metrics = new Array[Long](MetricGetter.values.length) - metrics(0) = -1 - - /** - * Compare the specified memory values with the saved peak executor memory - * values, and update if there is a new peak value. - * - * @param executorMetrics the executor metrics to compare - * @return if there is a new peak value for any metric - */ - def compareAndUpdate(executorMetrics: Array[Long]): Boolean = { - var updated: Boolean = false - - (0 until MetricGetter.values.length).foreach { metricIdx => - val newVal = executorMetrics(metricIdx) - if ( newVal > metrics(metricIdx)) { - updated = true - metrics(metricIdx) = newVal - } - } - updated - } -} 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 ff908ec8fff5..293e8369677f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -26,7 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo import org.apache.spark.{SparkConf, TaskEndReason} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.ui.SparkUI @@ -166,7 +166,7 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends case class SparkListenerExecutorMetricsUpdate( execId: String, accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])], - executorUpdates: Option[Array[Long]] = None) + executorUpdates: Option[ExecutorMetrics] = None) extends SparkListenerEvent /** @@ -182,7 +182,7 @@ case class SparkListenerStageExecutorMetrics( execId: String, stageId: Int, stageAttemptId: Int, - executorMetrics: Array[Long]) + executorMetrics: ExecutorMetrics) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 4d80eaadd41a..037e083c4370 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -76,7 +77,7 @@ private[spark] trait TaskScheduler { execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorUpdates: Array[Long]): Boolean + executorUpdates: ExecutorMetrics): Boolean /** * Get an application ID associated with the job. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index df1c13ebf8ff..cd9bc5f51e1d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -28,6 +28,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -443,7 +444,7 @@ private[spark] class TaskSchedulerImpl( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: Array[Long]): Boolean = { + executorMetrics: ExecutorMetrics): Boolean = { // (taskId, stageId, stageAttemptId, accumUpdates) val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized { accumUpdates.flatMap { case (id, updates) => 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 b182877d416e..836cef905aa8 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1 @@ -690,9 +690,9 @@ private[spark] class AppStatusListener( // check if there is a new peak value for any of the executor level memory metrics // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed // for the live UI. - event.executorUpdates.foreach { updates: Array[Long] => + event.executorUpdates.foreach { updates: ExecutorMetrics => liveExecutors.get(event.execId).foreach { exec: LiveExecutor => - if (exec.peakExecutorMetrics.compareAndUpdate(updates)) { + if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(updates)) { maybeUpdate(exec, now) } } @@ -708,7 +708,7 @@ private[spark] class AppStatusListener( liveExecutors.get(executorMetrics.execId) .orElse(deadExecutors.get(executorMetrics.execId)) match { case Some(exec) => - if (exec.peakExecutorMetrics.compareAndUpdate(executorMetrics.executorMetrics)) { + if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) { maybeUpdate(exec, now) } case None => 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 c747ce66b78c..9c1ceb3a2559 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -26,8 +26,8 @@ import scala.collection.mutable.HashMap import com.google.common.collect.Interners import org.apache.spark.JobExecutionStatus -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, PeakExecutorMetrics, StageInfo, TaskInfo} +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, TaskInfo} import org.apache.spark.status.api.v1 import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.SparkUI @@ -269,7 +269,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE def hasMemoryInfo: Boolean = totalOnHeap >= 0L // peak values for executor level metrics - val peakExecutorMetrics = new PeakExecutorMetrics + val peakExecutorMetrics = new ExecutorMetrics() def hostname: String = if (host != null) host else hostPort.split(":")(0) @@ -306,7 +306,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE executorLogs, memoryMetrics, blacklistedInStages, - if (peakExecutorMetrics.metrics(0) == -1) None else Some(peakExecutorMetrics.metrics)) + if (peakExecutorMetrics.isSet()) Some(peakExecutorMetrics) else None) 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 20c89b054485..220be3ab141c 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 @@ -28,7 +28,8 @@ import com.fasterxml.jackson.databind.{DeserializationContext, JsonDeserializer, import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize} import org.apache.spark.JobExecutionStatus -import org.apache.spark.metrics.MetricGetter +import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.metrics.ExecutorMetricType case class ApplicationInfo private[spark]( id: String, @@ -103,9 +104,9 @@ class ExecutorSummary private[spark]( val executorLogs: Map[String, String], val memoryMetrics: Option[MemoryMetrics], val blacklistedInStages: Set[Int], - @JsonSerialize(using = classOf[PeakMemoryMetricsSerializer]) - @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer]) - val peakMemoryMetrics: Option[Array[Long]]) + @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) + @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) + val peakMemoryMetrics: Option[ExecutorMetrics]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, @@ -113,32 +114,34 @@ class MemoryMetrics private[spark]( val totalOnHeapStorageMemory: Long, val totalOffHeapStorageMemory: Long) -/** deserializer for peakMemoryMetrics: convert to array ordered by metric name */ -private class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] { +/** deserializer for peakMemoryMetrics: convert map to ExecutorMetrics */ +private[spark] class ExecutorMetricsJsonDeserializer + extends JsonDeserializer[Option[ExecutorMetrics]] { override def deserialize( jsonParser: JsonParser, - deserializationContext: DeserializationContext): Option[Array[Long]] = { + deserializationContext: DeserializationContext): Option[ExecutorMetrics] = { val metricsMap = jsonParser.readValueAs[Option[Map[String, Long]]]( new TypeReference[Option[Map[String, java.lang.Long]]] {}) metricsMap match { case Some(metrics) => - Some(MetricGetter.values.map(m => metrics.getOrElse(m.name, 0L)).toArray) + Some(new ExecutorMetrics(metrics)) case None => None } } } -/** serializer for peakMemoryMetrics: convert array to map with metric name as key */ -private class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] { +/** serializer for peakMemoryMetrics: convert ExecutorMetrics to map with metric name as key */ +private[spark] class ExecutorMetricsJsonSerializer + extends JsonSerializer[Option[ExecutorMetrics]] { override def serialize( - metrics: Option[Array[Long]], + metrics: Option[ExecutorMetrics], jsonGenerator: JsonGenerator, serializerProvider: SerializerProvider): Unit = { metrics match { case Some(m) => - val metricsMap = (0 until MetricGetter.values.length).map { idx => - MetricGetter.values (idx).name -> m(idx) + val metricsMap = ExecutorMetricType.values.map { metricType => + metricType.name -> m.getMetricValue(metricType) }.toMap - jsonGenerator.writeObject (metricsMap) + jsonGenerator.writeObject(metricsMap) case None => } } 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 34da386a4f17..16613c3cc1e4 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -31,7 +31,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark._ import org.apache.spark.executor._ -import org.apache.spark.metrics.MetricGetter +import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -392,11 +392,11 @@ private[spark] object JsonProtocol { ("Updated Blocks" -> updatedBlocks) } - /** Convert executor metrics (indexed by MetricGetter.values) to JSON. */ - def executorMetricsToJson(executorMetrics: Array[Long]): JValue = { - val metrics = MetricGetter.idxAndValues.map { case (idx, metric) => - JField(metric.name, executorMetrics(idx)) - } + /** Convert executor metrics to JSON. */ + def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { + val metrics = ExecutorMetricType.values.map{ metricType => + JField(metricType.name, executorMetrics.getMetricValue(metricType)) + } JObject(metrics: _*) } @@ -608,13 +608,13 @@ private[spark] object JsonProtocol { SparkListenerTaskGettingResult(taskInfo) } - /** Extract the executor metrics (indexed by MetricGetter.values) from JSON. */ - def executorMetricsFromJson(json: JValue): Array[Long] = { + /** Extract the executor metrics from JSON. */ + def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val metrics = - MetricGetter.values.map {metric => - val metricVal = jsonOption(json \ metric.name).map(_.extract[Long]).getOrElse(0L) - metricVal} - metrics.toArray + ExecutorMetricType.values.map { metric => + metric.name -> jsonOption(json \ metric.name).map(_.extract[Long]).getOrElse(0L) + }.toMap + new ExecutorMetrics(metrics) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 341ab2e8975a..c7547d9ffcf1 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -29,7 +29,7 @@ import org.mockito.Matchers._ import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -213,8 +213,8 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) - val executorUpdates = Array(123456L, 543L, 12345L, 1234L, 123L, - 12L, 432L, 321L, 654L, 765L) + val executorUpdates = new ExecutorMetrics(Array(123456L, 543L, 12345L, 1234L, 123L, + 12L, 432L, 321L, 654L, 765L)) val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( Heartbeat(executorId, Array(1L -> metrics.accumulators()), blockManagerId, executorUpdates)) if (executorShouldReregister) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 30fca1cb3fee..db5ad7f30a04 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -30,6 +30,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} @@ -121,7 +122,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorUpdates: Array[Long]): Boolean = true + executorUpdates: ExecutorMetrics): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -636,7 +637,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: Array[Long]): Boolean = true + executorMetrics: ExecutorMetrics): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None 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 4e3dd054ce50..98cd7be53820 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -31,10 +31,10 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.io._ -import org.apache.spark.metrics.{MetricGetter, MetricsSystem} +import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} @@ -277,16 +277,16 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit Map( ((0, "1"), new SparkListenerStageExecutorMetrics("1", 0, 0, - Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))), + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), ((0, "2"), new SparkListenerStageExecutorMetrics("2", 0, 0, - Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))), + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), ((1, "1"), new SparkListenerStageExecutorMetrics("1", 1, 0, - Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))), + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), ((1, "2"), new SparkListenerStageExecutorMetrics("2", 1, 0, - Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) // Events to post. val events = Array( @@ -298,52 +298,53 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)), + new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), createExecutorMetricsUpdateEvent(2, - Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)), + new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)), + new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 createExecutorMetricsUpdateEvent(2, - Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)), + new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 createExecutorMetricsUpdateEvent(1, - Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)), + new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(2, - Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)), + new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), - // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 + // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks createExecutorMetricsUpdateEvent(1, - Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 + new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9; + // initialize stage 1 peaks createExecutorMetricsUpdateEvent(2, - Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)), + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 createExecutorMetricsUpdateEvent(1, - Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)), - // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 + new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), + // enew ExecutorMetrics(xec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 createExecutorMetricsUpdateEvent(2, - Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)), + new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 createExecutorMetricsUpdateEvent(1, - Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)), + new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), // exec 2: new stage 1 peak for metrics at index: 7 createExecutorMetricsUpdateEvent(2, - Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)), + new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, - Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)), + new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(2, - Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)), + new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -411,7 +412,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit private def createExecutorMetricsUpdateEvent( executorId: Int, - executorMetrics: Array[Long]): SparkListenerExecutorMetricsUpdate = { + executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { val taskMetrics = TaskMetrics.empty taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) @@ -455,11 +456,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(executorMetrics.execId === expectedMetrics.execId) assert(executorMetrics.stageId === expectedMetrics.stageId) assert(executorMetrics.stageAttemptId === expectedMetrics.stageAttemptId) - assert(executorMetrics.executorMetrics.length === - expectedMetrics.executorMetrics.length) - (0 until MetricGetter.values.length).foreach { idx => - assert(executorMetrics.executorMetrics(idx) === - expectedMetrics.executorMetrics(idx)) + ExecutorMetricType.values.foreach { metricType => + assert(executorMetrics.executorMetrics.getMetricValue(metricType) === + expectedMetrics.executorMetrics.getMetricValue(metricType)) } case None => assert(false) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 4ffd3430b50a..149e58fc5a90 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -90,5 +91,5 @@ private class DummyTaskScheduler extends TaskScheduler { execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: Array[Long]): Boolean = true + executorMetrics: ExecutorMetrics): Boolean = true } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index e24d550a6266..f3eb986ff50f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -22,6 +22,7 @@ import java.net.URI import java.util.concurrent.atomic.AtomicInteger import org.apache.hadoop.fs.Path +import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ import org.scalatest.BeforeAndAfter @@ -217,7 +218,12 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp // Verify the same events are replayed in the same order assert(sc.eventLogger.isDefined) - val originalEvents = sc.eventLogger.get.loggedEvents + val originalEvents = sc.eventLogger.get.loggedEvents.filter { e => + JsonProtocol.sparkEventFromJson(e) match { + case event: SparkListenerStageExecutorMetrics => false + case _ => true + } + } val replayedEvents = eventMonster.loggedEvents originalEvents.zip(replayedEvents).foreach { case (e1, e2) => // Don't compare the JSON here because accumulators in StageInfo may be out of order 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 47ed6efb5f4e..f352ff6e457a 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -28,8 +28,8 @@ import scala.reflect.{classTag, ClassTag} import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.metrics.MetricGetter +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.api.v1 @@ -1270,8 +1270,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // expected peak values for each executor val expectedValues = Map( - "1" -> Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), - "2" -> Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) // check that the stored peak values match the expected values expectedValues.foreach { case (id, metrics) => @@ -1279,8 +1279,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - (0 until MetricGetter.values.length).foreach { idx => - assert(actual(idx) === metrics(idx)) + ExecutorMetricType.values.foreach { metricType => + assert(actual.getMetricValue(metricType) === metrics.getMetricValue(metricType)) } case _ => assert(false) @@ -1299,23 +1299,23 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onStageSubmitted(createStageSubmittedEvent(0)) listener.onStageSubmitted(createStageSubmittedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 0, 0, - Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))) + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 0, 0, - Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))) listener.onStageCompleted(createStageCompletedEvent(0)) // executor 1 is removed before stage 1 has finished, the stage executor metrics // are logged afterwards and should still be used to update the executor metrics. listener.onExecutorRemoved(createExecutorRemovedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 1, 0, - Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 1, 0, - Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))) + new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L), - "2" -> Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L)) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) // check that the stored peak values match the expected values for ((id, metrics) <- expectedValues) { @@ -1323,8 +1323,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - (0 until MetricGetter.values.length).foreach { idx => - assert(actual(idx) === metrics(idx)) + ExecutorMetricType.values.foreach { metricType => + assert(actual.getMetricValue(metricType) === metrics.getMetricValue(metricType)) } case _ => assert(false) @@ -1400,6 +1400,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { taskMetrics.incDiskBytesSpilled(111) taskMetrics.incMemoryBytesSpilled(222) val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo))) - SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics)) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, + Some(new ExecutorMetrics(executorMetrics))) } } 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 5f5129ad051c..1e0d2af9a471 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -30,7 +30,7 @@ import org.scalatest.exceptions.TestFailedException import org.apache.spark._ import org.apache.spark.executor._ -import org.apache.spark.metrics.MetricGetter +import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -95,7 +95,8 @@ class JsonProtocolSuite extends SparkFunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } - val executorUpdates = Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L) + val executorUpdates = new ExecutorMetrics( + Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), Some(executorUpdates)) } @@ -104,7 +105,7 @@ class JsonProtocolSuite extends SparkFunSuite { "In your multitude...", 300), RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100L, 0L)) val stageExecutorMetrics = SparkListenerStageExecutorMetrics("1", 2, 3, - Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) + new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L))) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -439,12 +440,14 @@ class JsonProtocolSuite extends SparkFunSuite { test("executorMetricsFromJson backward compatibility: handle missing metrics") { // any missing metrics should be set to 0 - val executorMetrics = Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 789L) + val executorMetrics = new ExecutorMetrics( + Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 789L)) val oldExecutorMetricsJson = JsonProtocol.executorMetricsToJson(executorMetrics) .removeField( _._1 == "MappedPoolMemory") - val exepectedExecutorMetrics = Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 0L) - assertExecutorMetricsEquals(exepectedExecutorMetrics, + val expectedExecutorMetrics = new ExecutorMetrics( + Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 0L)) + assertEquals(expectedExecutorMetrics, JsonProtocol.executorMetricsFromJson(oldExecutorMetricsJson)) } @@ -593,12 +596,13 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(stageAttemptId1 === stageAttemptId2) assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) }) - assertExecutorMetricsEquals(e1.executorUpdates, e2.executorUpdates) + assertOptionEquals(e1.executorUpdates, e2.executorUpdates, + (e1: ExecutorMetrics, e2: ExecutorMetrics) => assertEquals(e1, e2)) case (e1: SparkListenerStageExecutorMetrics, e2: SparkListenerStageExecutorMetrics) => assert(e1.execId === e2.execId) assert(e1.stageId === e2.stageId) assert(e1.stageAttemptId === e2.stageAttemptId) - assertExecutorMetricsEquals(e1.executorMetrics, e2.executorMetrics) + assertEquals(e1.executorMetrics, e2.executorMetrics) case (e1, e2) => assert(e1 === e2) case _ => fail("Events don't match in types!") @@ -749,6 +753,12 @@ private[spark] object JsonProtocolSuite extends Assertions { assertStackTraceElementEquals) } + private def assertEquals(metrics1: ExecutorMetrics, metrics2: ExecutorMetrics) { + ExecutorMetricType.values.foreach { metricType => + assert(metrics1.getMetricValue(metricType) === metrics2.getMetricValue(metricType)) + } + } + private def assertJsonStringEquals(expected: String, actual: String, metadata: String) { val expectedJson = pretty(parse(expected)) val actualJson = pretty(parse(actual)) @@ -799,26 +809,6 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(ste1 === ste2) } - private def assertExecutorMetricsEquals( - metrics1: Option[Array[Long]], - metrics2: Option[Array[Long]]) { - (metrics1, metrics2) match { - case (Some(m1), Some(m2)) => - assertExecutorMetricsEquals(m1, m2) - case (None, None) => - case _ => - assert(false) - } - } - - private def assertExecutorMetricsEquals(metrics1: Array[Long], metrics2: Array[Long]) { - assert(metrics1.length === MetricGetter.values.length) - assert(metrics2.length === MetricGetter.values.length) - (0 until MetricGetter.values.length).foreach { idx => - assert(metrics1(idx) === metrics2(idx)) - } - } - /** ----------------------------------- * | Util methods for constructing events | * ------------------------------------ */ @@ -887,7 +877,7 @@ private[spark] object JsonProtocolSuite extends Assertions { } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L)) + Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L))) } else { None } From 04875b8c8ee2f181103331c2cc9544b75b854db6 Mon Sep 17 00:00:00 2001 From: Reza Safi Date: Thu, 26 Jul 2018 17:33:52 -0400 Subject: [PATCH 22/29] Integration of ProcessTreeMetrics with PR 21221 Some improvements in integration Integration with the unit tests of the upstream open PR Fix an isuue with memory info computation. Fix scalastyle errors Some changes to address comments --- .../spark/executor/ProcessTreeMetrics.scala | 28 + .../spark/executor/ProcfsBasedSystems.scala | 237 +++++++ .../apache/spark/memory/MemoryManager.scala | 7 + .../spark/metrics/ExecutorMetricType.scala | 14 + ...ith_executor_metrics_json_expectation.json | 10 + .../application_1506645932520_24630151 | 18 +- .../scheduler/EventLoggingListenerSuite.scala | 67 +- .../spark/status/AppStatusListenerSuite.scala | 71 +- .../apache/spark/util/JsonProtocolSuite.scala | 667 +++++++++--------- 9 files changed, 715 insertions(+), 404 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ProcessTreeMetrics.scala create mode 100644 core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala diff --git a/core/src/main/scala/org/apache/spark/executor/ProcessTreeMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ProcessTreeMetrics.scala new file mode 100644 index 000000000000..87d1356890e6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ProcessTreeMetrics.scala @@ -0,0 +1,28 @@ +/* + * 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.executor + +private[spark] trait ProcessTreeMetrics { + def isAvailable: Boolean + def pid: Int + def computePid(): Int + def createProcessTree() + def updateProcessTree() + def getRSSInfo(): Long + def getVirtualMemInfo(): Long +} diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala new file mode 100644 index 000000000000..1be94a644f48 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala @@ -0,0 +1,237 @@ +/* + * 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.executor + +import java.io._ +import java.nio.charset.Charset +import java.nio.file.{Files, Paths} + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.Queue + +import org.apache.spark.internal.Logging + + +// Some of the ideas here are taken from the ProcfsBasedProcessTree class in hadoop +// project. +class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { + val procfsDir = "/proc/" + var isAvailable: Boolean = isItProcfsBased + val pid: Int = computePid() + val ptree: scala.collection.mutable.Map[ Int, Set[Int]] = + scala.collection.mutable.Map[ Int, Set[Int]]() + val PROCFS_STAT_FILE = "stat" + + createProcessTree + + def isItProcfsBased: Boolean = { + val testing = sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + if (testing) { + return true + } + try { + if (!Files.exists(Paths.get(procfsDir))) { + return false + } + } + catch { + case f: FileNotFoundException => return false + } + true + } + + + def computePid(): Int = { + if (!isAvailable) { + return -1; + } + try { + val cmd = Array("bash", "-c", "echo $PPID") + val length = 10 + var out: Array[Byte] = Array.fill[Byte](length)(0) + Runtime.getRuntime.exec(cmd).getInputStream.read(out) + val pid = Integer.parseInt(new String(out, "UTF-8").trim) + return pid; + } + catch { + case e: IOException => logDebug("IO Exception when trying to compute process tree." + + " As a result reporting of ProcessTree metrics is stopped") + isAvailable = false + return -1 + case _ => logDebug("Some exception occurred when trying to compute process tree. " + + "As a result reporting of ProcessTree metrics is stopped") + isAvailable = false + return -1 + } + } + + + def createProcessTree(): Unit = { + if (!isAvailable) { + return + } + val queue: Queue[Int] = new Queue[Int]() + queue += pid + while( !queue.isEmpty ) { + val p = queue.dequeue() + val c = getChildPIds(p) + if(!c.isEmpty) { + queue ++= c + ptree += (p -> c.toSet) + } + else { + ptree += (p -> Set[Int]()) + } + } + } + + + def updateProcessTree(): Unit = { + if (!isAvailable) { + return + } + val queue: Queue[Int] = new Queue[Int]() + queue += pid + while( !queue.isEmpty ) { + val p = queue.dequeue() + val c = getChildPIds(p) + if(!c.isEmpty) { + queue ++= c + val preChildren = ptree.get(p) + preChildren match { + case Some(children) => if (!c.toSet.equals(children)) { + val diff: Set[Int] = children -- c.toSet + ptree.update(p, c.toSet ) + diff.foreach(ptree.remove(_)) + } + case None => ptree.update(p, c.toSet ) + } + } + else { + ptree.update(p, Set[Int]()) + } + } + } + + + /** + * Hadoop ProcfsBasedProcessTree class used regex and pattern matching to retrive the memory + * info. I tried that but found it not correct during tests, so I used normal string analysis + * instead. The computation of RSS and Vmem are based on proc(5): + * http://man7.org/linux/man-pages/man5/proc.5.html + */ + def getProcessInfo(pid: Int): String = { + try { + val pidDir: File = new File(procfsDir, pid.toString) + val fReader = new InputStreamReader( + new FileInputStream( + new File(pidDir, PROCFS_STAT_FILE)), Charset.forName("UTF-8")) + val in: BufferedReader = new BufferedReader(fReader) + val procInfo = in.readLine + in.close + fReader.close + return procInfo + } catch { + case f: FileNotFoundException => return null + } + null + } + + + def getRSSInfo(): Long = { + if (!isAvailable) { + return -1 + } + updateProcessTree + val pids = ptree.keySet + var totalRss = 0L + for (p <- pids) { + totalRss += getProcessRSSInfo(p) + } + totalRss + } + + def getProcessRSSInfo(pid: Int): Long = { + val pInfo = getProcessInfo(pid) + if (pInfo != null) { + val pInfoSplit = pInfo.split(" ") + // According to proc(5) RSS is the 24th value when we read first line of /proc/[pid]/stat + return pInfoSplit(23).toLong + } + 0 + } + + def getVirtualMemInfo(): Long = { + if (!isAvailable) { + return -1 + } + // We won't call updateProcessTree here since we already did that when we + // computed RSS info + val pids = ptree.keySet + var totalVMem = 0L + for (p <- pids) { + totalVMem += getProcessVirtualMemInfo(p) + } + totalVMem + } + + + def getProcessVirtualMemInfo(pid: Int): Long = { + val pInfo = getProcessInfo(pid) + if (pInfo != null) { + val pInfoSplit = pInfo.split(" ") + // According to proc(5) Vmem is the 23rd value when we read first line of /proc/[pid]/stat + return pInfoSplit(22).toLong + } + 0L + } + + + def getChildPIds(pid: Int): ArrayBuffer[Int] = { + try { + val cmd = Array("pgrep", "-P", pid.toString) + val input = Runtime.getRuntime.exec(cmd).getInputStream + val childPidsInByte: mutable.ArrayBuffer[Byte] = new mutable.ArrayBuffer() + var d = input.read() + while (d != -1) { + childPidsInByte.append(d.asInstanceOf[Byte]) + d = input.read() + } + input.close() + val childPids = new String(childPidsInByte.toArray, "UTF-8").split("\n") + val childPidsInInt: ArrayBuffer[Int] = new ArrayBuffer[Int]() + for (p <- childPids) { + if (p != "") { + childPidsInInt += Integer.parseInt(p) + } + } + childPidsInInt + } catch { + case e: IOException => logDebug("IO Exception when trying to compute process tree." + + " As a result reporting of ProcessTree metrics is stopped") + isAvailable = false + return new mutable.ArrayBuffer() + case _ => logDebug("Some exception occurred when trying to compute process tree. As a result" + + " reporting of ProcessTree metrics is stopped") + isAvailable = false + return new mutable.ArrayBuffer() + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 965515b8abe1..92a59b20417c 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -20,6 +20,7 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy import org.apache.spark.SparkConf +import org.apache.spark.executor.{ProcessTreeMetrics, ProcfsBasedSystems} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.storage.BlockId @@ -200,6 +201,12 @@ private[spark] abstract class MemoryManager( */ final def offHeapStorageMemoryUsed: Long = offHeapStorageMemoryPool.memoryUsed + /** + * If the system isn't procfsBased the process tree metrics' values will be -1, + * meaning not available + */ + final val pTreeInfo: ProcessTreeMetrics = new ProcfsBasedSystems + /** * Returns the execution memory consumption, in bytes, for the given task. */ diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index 456663e017c2..23c2d18359bf 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -59,6 +59,18 @@ case object JVMOffHeapMemory extends ExecutorMetricType { } } +case object ProcessTreeRSSMemory extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { + memoryManager.pTreeInfo.getRSSInfo() + } +} + +case object ProcessTreeVMemory extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { + memoryManager.pTreeInfo.getVirtualMemInfo() + } +} + case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType( _.onHeapExecutionMemoryUsed) @@ -88,6 +100,8 @@ private[spark] object ExecutorMetricType { val values = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, + ProcessTreeRSSMemory, + ProcessTreeVMemory, OnHeapExecutionMemory, OffHeapExecutionMemory, OnHeapStorageMemory, 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 9bf2086cc8e7..03735714db05 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 @@ -30,6 +30,8 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 905801, "JVMOffHeapMemory" : 205304696, + "ProcessTreeRSSMemory": 200000000, + "ProcessTreeVMemory": 500000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 905801, "OnHeapExecutionMemory" : 0, @@ -170,6 +172,8 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 63104457, "JVMOffHeapMemory" : 95657456, + "ProcessTreeRSSMemory": 100000000, + "ProcessTreeVMemory": 500000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 100853193, "OnHeapExecutionMemory" : 37748736, @@ -214,6 +218,8 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 69535048, "JVMOffHeapMemory" : 90709624, + "ProcessTreeRSSMemory": 200000000, + "ProcessTreeVMemory": 600000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 69535048, "OnHeapExecutionMemory" : 0, @@ -258,6 +264,8 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 58468944, "JVMOffHeapMemory" : 91208368, + "ProcessTreeRSSMemory": 90000000, + "ProcessTreeVMemory": 400000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 58468944, "OnHeapExecutionMemory" : 0, @@ -302,6 +310,8 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 47962185, "JVMOffHeapMemory" : 100519936, + "ProcessTreeRSSMemory": 100000000, + "ProcessTreeVMemory": 500000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 47962185, "OnHeapExecutionMemory" : 0, diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 index c48ed741c56e..b6d9b759a920 100644 --- a/core/src/test/resources/spark-events/application_1506645932520_24630151 +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -33,11 +33,11 @@ {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144246,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"1920974","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"3562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"38","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1813","Value":"1812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"195602","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"3563","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1558","Value":"1557","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":3563,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":36845111,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":27318908,"Value":27318908,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":3562,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":349287,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":33,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1498974375,"Value":1498974375,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1922,"Value":1922,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":49547405,"Value":49547405,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":56,"Executor Deserialize CPU Time":49547405,"Executor Run Time":1922,"Executor CPU Time":1498974375,"Result Size":2394,"JVM GC Time":33,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":27318908,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144444,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"23365","Value":"63634","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":105575962,"Value":301246724,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":326,"Value":1107,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11931694025,"Value":31991331624,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13454,"Value":36578,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":531799977,"Value":1633060096,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":594,"Value":1930,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":594,"Executor Deserialize CPU Time":531799977,"Executor Run Time":13454,"Executor CPU Time":11931694025,"Result Size":1856,"JVM GC Time":326,"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":22950296,"Shuffle Write Time":105575962,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144840,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"24254","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":106963069,"Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":342,"Value":1449,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":12267596062,"Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13858,"Value":50436,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":519573839,"Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":573,"Value":2503,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":573,"Executor Deserialize CPU Time":519573839,"Executor Run Time":13858,"Executor CPU Time":12267596062,"Result Size":1856,"JVM GC Time":342,"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":31362123,"Shuffle Write Time":106963069,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":592412824,"JVMOffHeapMemory":202907152,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":355389,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":523121272,"JVMOffHeapMemory":88280720,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":52050147,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":52050147,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":214174608,"JVMOffHeapMemory":91548704,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47399168,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47399168,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":592412824,"JVMOffHeapMemory":202907152,"ProcessTreeRSSMemory":90000000,"ProcessTreeVMemory":500000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":355389,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":523121272,"JVMOffHeapMemory":88280720,"ProcessTreeRSSMemory":90000000,"ProcessTreeVMemory":400000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":52050147,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":52050147,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":214174608,"JVMOffHeapMemory":91548704,"ProcessTreeRSSMemory":85000000,"ProcessTreeVMemory":450000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47399168,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47399168,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"ProcessTreeRSSMemory":95000000,"ProcessTreeVMemory":480000000,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"ProcessTreeRSSMemory":200000000,"ProcessTreeVMemory":600000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Completion Time":1524182144852,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":2503,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":50436,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":1449,"Internal":true,"Count Failed Values":true}]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145327,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"49","Value":"86","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"2002","Value":"3814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1755","Value":"3312","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":32035583,"Value":59354491,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4788,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1785119941,"Value":3284094316,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":2182,"Value":4104,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":71500541,"Value":121047946,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":136,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":136,"Executor Deserialize CPU Time":71500541,"Executor Run Time":2182,"Executor CPU Time":1785119941,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349006,"Shuffle Write Time":32035583,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145971,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"5212268","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"9572","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"121634812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"9","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1703","Value":"5517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"525948","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"9573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1609","Value":"4921","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":9573,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":97944567,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":20055909,"Value":79410400,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":9572,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":941007,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":121634816,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":7182,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":896878991,"Value":4180973307,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1722,"Value":5826,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2787355,"Value":123835301,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":195,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2787355,"Executor Run Time":1722,"Executor CPU Time":896878991,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":242714,"Shuffle Write Time":20055909,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} @@ -46,10 +46,10 @@ {"Event":"SparkListenerExecutorAdded","Timestamp":1524182149826,"Executor ID":"7","Executor Info":{"Host":"node6340.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096","stderr":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"node6340.grid.company.com","Port":5933},"Maximum Memory":956615884,"Timestamp":1524182149983,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182152418,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"48","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"6093","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"5951","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":49790497,"Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":920,"Value":1015,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9619,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5299274511,"Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":7847,"Value":13673,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687811857,"Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1037,"Value":1232,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1037,"Executor Deserialize CPU Time":687811857,"Executor Run Time":7847,"Executor CPU Time":5299274511,"Result Size":2437,"JVM GC Time":920,"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":355051,"Shuffle Write Time":49790497,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":629553808,"JVMOffHeapMemory":205304696,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":397602,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":595946552,"JVMOffHeapMemory":91208368,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":58468944,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":58468944,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":755008624,"JVMOffHeapMemory":100519936,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47962185,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47962185,"OffHeapUnifiedMemory":0,"DirectPoolMemory":98230,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":629553808,"JVMOffHeapMemory":205304696,"ProcessTreeRSSMemory":200000000,"ProcessTreeVMemory":400000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":397602,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":595946552,"JVMOffHeapMemory":91208368,"ProcessTreeRSSMemory":80000000,"ProcessTreeVMemory":300000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":58468944,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":58468944,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":755008624,"JVMOffHeapMemory":100519936,"ProcessTreeRSSMemory":100000000,"ProcessTreeVMemory":500000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47962185,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47962185,"OffHeapUnifiedMemory":0,"DirectPoolMemory":98230,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"ProcessTreeRSSMemory": 100000000,"ProcessTreeVMemory": 500000000,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} {"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Completion Time":1524182152419,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":1232,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":13673,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":1015,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9619,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} 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 98cd7be53820..f7f1972ed6a2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -277,16 +277,20 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit Map( ((0, "1"), new SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), + new ExecutorMetrics(Array(5000L, 50L, 4000L, 8000L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L)))), ((0, "2"), new SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), + new ExecutorMetrics(Array(7000L, 70L, 4000L, 9000L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L)))), ((1, "1"), new SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), + new ExecutorMetrics(Array(7000L, 70L, 3000L, 5000L, 50L, 30L, 60L, 30L, 80L, 55L, + 50L, 0L)))), ((1, "2"), new SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) + new ExecutorMetrics(Array(7000L, 70L, 5000L, 8000L, 50L, 40L, 10L, 30L, 50L, 60L, + 40L, 40L))))) // Events to post. val events = Array( @@ -298,53 +302,58 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), + new ExecutorMetrics(Array(4000L, 50L, 3500L, 7500L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), - // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 + new ExecutorMetrics(Array(1500L, 50L, 3500L, 8500L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), + // exec 1: new stage 0 peaks for metrics at indexes: 2, 3, 4, 6, 8 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 + new ExecutorMetrics(Array(4000L, 50L, 4000L, 8000L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 2, 3, 6, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), - // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 + new ExecutorMetrics(Array(2000L, 50L, 4000L, 9000L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), + // exec 1: new stage 0 peaks for metrics at indexes: 7, 9 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 + new ExecutorMetrics(Array(2000L, 40L, 3500L, 8000L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 7, 8, 9, 10 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), + new ExecutorMetrics(Array(3500L, 50L, 3500L, 8500L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), - // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks + // exec 1: new stage 0 peaks for metrics at indexes: 0, 5, 9; initialize stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9; + new ExecutorMetrics(Array(5000L, 30L, 3000L, 5000L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, + 0L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 4, 5, 8, 9, 11; // initialize stage 1 peaks createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), + new ExecutorMetrics(Array(7000L, 70L, 4000L, 8000L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, + 40L))), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), - // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 + // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 2, 3, 5 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), - // enew ExecutorMetrics(xec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 + new ExecutorMetrics(Array(6000L, 70L, 3000L, 5000L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), + // exec 2: new stage 1 peaks for metrics at indexes: 2, 3, 5, 6, 9, 10 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), - // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 + new ExecutorMetrics(Array(5500L, 30L, 5000L, 8000L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, + 20L))), + // exec 1: new stage 1 peaks for metrics at indexes: 0, 6, 7, 9 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), - // exec 2: new stage 1 peak for metrics at index: 7 + new ExecutorMetrics(Array(7000L, 70L, 2500L, 3000L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), + // exec 2: new stage 1 peak for metrics at index: 9 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), + new ExecutorMetrics(Array(5500L, 40L, 3000L, 7000L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, + 20L))), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), + new ExecutorMetrics(Array(5500L, 70L, 2500L, 4000L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, + 0L))), createExecutorRemovedEvent(1), - // exec 2: new stage 1 peak for metrics at index: 6 + // exec 2: new stage 1 peak for metrics at index: 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), + new ExecutorMetrics(Array(4000L, 20L, 4000L, 7000L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) 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 f352ff6e457a..6c9b3a87da22 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1220,58 +1220,60 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + Array(4000L, 50L, 3500L, 7500L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) - // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 + Array(1500L, 50L, 3500L, 8500L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + // exec 1: new stage 0 peaks for metrics at indexes: 2, 3, 4, 6, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) - // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 + Array(4000L, 50L, 4000L, 8000L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 2, 3, 6, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) - // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 + Array(2000L, 50L, 4000L, 9000L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + // exec 1: new stage 0 peaks for metrics at indexes: 7, 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) - // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 + Array(2000L, 40L, 3500L, 8000L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 7, 8, 9, 10 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + Array(3500L, 50L, 3500L, 8500L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks listener.onStageSubmitted(createStageSubmittedEvent(1)) - // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 + // exec 1: new stage 0 peaks for metrics at indexes: 0, 5, 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) - // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 + Array(5000L, 30L, 3000L, 5000L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 4, 5, 8, 9, 11 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + Array(7000L, 80L, 4000L, 8000L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) // complete stage 0, and 3 more updates for each executor with just // stage 1 running listener.onStageCompleted(createStageCompletedEvent(0)) - // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 + // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 5 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) - // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 + Array(6000L, 70L, 3000L, 5000L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + // exec 2: new stage 1 peaks for metrics at indexes: 5, 6, 9, 10 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) - // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 + Array(5500L, 30L, 5000L, 8000L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + // exec 1: new stage 1 peaks for metrics at indexes: 0, 6, 7, 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) - // exec 2: new stage 1 peak for metrics at index: 7 + Array(7000L, 70L, 2500L, 3000L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + // exec 2: new stage 1 peak for metrics at index: 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + Array(5500L, 40L, 3000L, 7000L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) // exec 1: no new stage 1 peaks listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + Array(5500L, 70L, 2500L, 4000L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) listener.onExecutorRemoved(createExecutorRemovedEvent(1)) - // exec 2: new stage 1 peak for metrics at index: 6 + // exec 2: new stage 1 peak for metrics at index: 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + Array(4000L, 20L, 4000L, 7000L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 4000L, 8000L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 5000L, 9000L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L))) // check that the stored peak values match the expected values expectedValues.foreach { case (id, metrics) => @@ -1299,23 +1301,26 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onStageSubmitted(createStageSubmittedEvent(0)) listener.onStageSubmitted(createStageSubmittedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))) + new ExecutorMetrics(Array(5000L, 50L, 4000L, 8000L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))) + new ExecutorMetrics(Array(7000L, 70L, 4000L, 9000L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))) listener.onStageCompleted(createStageCompletedEvent(0)) // executor 1 is removed before stage 1 has finished, the stage executor metrics // are logged afterwards and should still be used to update the executor metrics. listener.onExecutorRemoved(createExecutorRemovedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))) + new ExecutorMetrics(Array(7000L, 70L, 3000L, 5000L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + new ExecutorMetrics(Array(7000L, 80L, 5000L, 8000L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 4000L, 8000L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 5000L, 9000L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L))) // check that the stored peak values match the expected values for ((id, metrics) <- expectedValues) { 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 1e0d2af9a471..5f1fe88187d1 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -92,11 +92,12 @@ class JsonProtocolSuite extends SparkFunSuite { val executorMetricsUpdate = { // Use custom accum ID for determinism val accumUpdates = - makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) - .accumulators().map(AccumulatorSuite.makeInfo) - .zipWithIndex.map { case (a, i) => a.copy(id = i) } + makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) + .accumulators().map(AccumulatorSuite.makeInfo) + .zipWithIndex.map { case (a, i) => a.copy(id = i) } val executorUpdates = new ExecutorMetrics( - Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) + Array(543L, 123456L, 123456L, 256912L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), Some(executorUpdates)) } @@ -105,8 +106,8 @@ class JsonProtocolSuite extends SparkFunSuite { "In your multitude...", 300), RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100L, 0L)) val stageExecutorMetrics = SparkListenerStageExecutorMetrics("1", 2, 3, - new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L))) - + new ExecutorMetrics(Array(543L, 123456L, 123456L, 256912L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L))) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) testEvent(taskStart, taskStartJsonString) @@ -232,7 +233,7 @@ class JsonProtocolSuite extends SparkFunSuite { hasHadoopInput = true, hasOutput = true, hasRecords = false) val newJson = JsonProtocol.taskMetricsToJson(metrics) val oldJson = newJson.removeField { case (field, _) => field == "Records Read" } - .removeField { case (field, _) => field == "Records Written" } + .removeField { case (field, _) => field == "Records Written" } val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) assert(newMetrics.inputMetrics.recordsRead == 0) assert(newMetrics.outputMetrics.recordsWritten == 0) @@ -244,7 +245,7 @@ class JsonProtocolSuite extends SparkFunSuite { hasHadoopInput = false, hasOutput = false, hasRecords = false) val newJson = JsonProtocol.taskMetricsToJson(metrics) val oldJson = newJson.removeField { case (field, _) => field == "Total Records Read" } - .removeField { case (field, _) => field == "Shuffle Records Written" } + .removeField { case (field, _) => field == "Shuffle Records Written" } val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) assert(newMetrics.shuffleReadMetrics.recordsRead == 0) assert(newMetrics.shuffleWriteMetrics.recordsWritten == 0) @@ -440,14 +441,14 @@ class JsonProtocolSuite extends SparkFunSuite { test("executorMetricsFromJson backward compatibility: handle missing metrics") { // any missing metrics should be set to 0 - val executorMetrics = new ExecutorMetrics( - Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 789L)) + val executorMetrics = new ExecutorMetrics(Array(12L, 23L, 20L, 40L, 45L, 67L, 78L, 89L, + 90L, 123L, 456L, 789L)) val oldExecutorMetricsJson = JsonProtocol.executorMetricsToJson(executorMetrics) .removeField( _._1 == "MappedPoolMemory") - val expectedExecutorMetrics = new ExecutorMetrics( - Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 0L)) - assertEquals(expectedExecutorMetrics, + val exepectedExecutorMetrics = new ExecutorMetrics(Array(12L, 23L, 20L, 40L, 45L, 67L, + 78L, 89L, 90L, 123L, 456L, 0L)) + assertEquals(exepectedExecutorMetrics, JsonProtocol.executorMetricsFromJson(oldExecutorMetricsJson)) } @@ -458,7 +459,7 @@ class JsonProtocolSuite extends SparkFunSuite { (TestBlockId("feebo"), BlockStatus(StorageLevel.DISK_ONLY, 3L, 4L))) val blocksJson = JArray(blocks.toList.map { case (id, status) => ("Block ID" -> id.toString) ~ - ("Status" -> JsonProtocol.blockStatusToJson(status)) + ("Status" -> JsonProtocol.blockStatusToJson(status)) }) testAccumValue(Some(RESULT_SIZE), 3L, JInt(3)) testAccumValue(Some(shuffleRead.REMOTE_BLOCKS_FETCHED), 2, JInt(2)) @@ -597,7 +598,7 @@ private[spark] object JsonProtocolSuite extends Assertions { assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) }) assertOptionEquals(e1.executorUpdates, e2.executorUpdates, - (e1: ExecutorMetrics, e2: ExecutorMetrics) => assertEquals(e1, e2)) + (e1: ExecutorMetrics, e2: ExecutorMetrics) => assertEquals(e1, e2)) case (e1: SparkListenerStageExecutorMetrics, e2: SparkListenerStageExecutorMetrics) => assert(e1.execId === e2.execId) assert(e1.stageId === e2.stageId) @@ -721,12 +722,12 @@ private[spark] object JsonProtocolSuite extends Assertions { case (r1: TaskKilled, r2: TaskKilled) => assert(r1.reason == r2.reason) case (TaskCommitDenied(jobId1, partitionId1, attemptNumber1), - TaskCommitDenied(jobId2, partitionId2, attemptNumber2)) => + TaskCommitDenied(jobId2, partitionId2, attemptNumber2)) => assert(jobId1 === jobId2) assert(partitionId1 === partitionId2) assert(attemptNumber1 === attemptNumber2) case (ExecutorLostFailure(execId1, exit1CausedByApp, reason1), - ExecutorLostFailure(execId2, exit2CausedByApp, reason2)) => + ExecutorLostFailure(execId2, exit2CausedByApp, reason2)) => assert(execId1 === execId2) assert(exit1CausedByApp === exit2CausedByApp) assert(reason1 === reason2) @@ -736,8 +737,8 @@ private[spark] object JsonProtocolSuite extends Assertions { } private def assertEquals( - details1: Map[String, Seq[(String, String)]], - details2: Map[String, Seq[(String, String)]]) { + details1: Map[String, Seq[(String, String)]], + details2: Map[String, Seq[(String, String)]]) { details1.zip(details2).foreach { case ((key1, values1: Seq[(String, String)]), (key2, values2: Seq[(String, String)])) => assert(key1 === key2) @@ -779,8 +780,7 @@ private[spark] object JsonProtocolSuite extends Assertions { } } - private def assertOptionEquals[T]( - opt1: Option[T], + private def assertOptionEquals[T](opt1: Option[T], opt2: Option[T], assertEquals: (T, T) => Unit) { if (opt1.isDefined) { @@ -795,8 +795,7 @@ private[spark] object JsonProtocolSuite extends Assertions { * Use different names for methods we pass in to assertSeqEquals or assertOptionEquals */ - private def assertBlocksEquals( - blocks1: Seq[(BlockId, BlockStatus)], + private def assertBlocksEquals(blocks1: Seq[(BlockId, BlockStatus)], blocks2: Seq[(BlockId, BlockStatus)]) = { assertSeqEquals(blocks1, blocks2, assertBlockEquals) } @@ -855,8 +854,7 @@ private[spark] object JsonProtocolSuite extends Assertions { taskInfo } - private def makeAccumulableInfo( - id: Int, + private def makeAccumulableInfo(id: Int, internal: Boolean = false, countFailedValues: Boolean = false, metadata: Option[String] = None): AccumulableInfo = @@ -864,10 +862,9 @@ private[spark] object JsonProtocolSuite extends Assertions { internal, countFailedValues, metadata) /** Creates an SparkListenerExecutorMetricsUpdate event */ - private def makeExecutorMetricsUpdate( - execId: String, - includeTaskMetrics: Boolean, - includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { + private def makeExecutorMetricsUpdate(execId: String, + includeTaskMetrics: Boolean, + includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { val taskMetrics = if (includeTaskMetrics) { Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), @@ -877,8 +874,9 @@ private[spark] object JsonProtocolSuite extends Assertions { } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L))) - } else { + Some(new ExecutorMetrics(Array(123456L, 543L, 123456L, 256912L, 0L, 0L, 0L, 0L, 0L, + 0L, 0L, 0L))) + } else { None } SparkListenerExecutorMetricsUpdate(execId, taskMetrics, executorMetricsUpdate) @@ -888,8 +886,7 @@ private[spark] object JsonProtocolSuite extends Assertions { * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is * set to true) or read data from a shuffle otherwise. */ - private def makeTaskMetrics( - a: Long, + private def makeTaskMetrics(a: Long, b: Long, c: Long, d: Long, @@ -946,44 +943,44 @@ private[spark] object JsonProtocolSuite extends Assertions { * ---------------------------------------- */ private val stageSubmittedJsonString = - """ - |{ - | "Event": "SparkListenerStageSubmitted", - | "Stage Info": { - | "Stage ID": 100, - | "Stage Attempt ID": 0, - | "Stage Name": "greetings", - | "Number of Tasks": 200, - | "RDD Info": [], - | "Parent IDs" : [100, 200, 300], - | "Details": "details", - | "Accumulables": [ - | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { - | "ID": 1, - | "Name": "Accumulable1", - | "Update": "delta1", - | "Value": "val1", - | "Internal": false, - | "Count Failed Values": false - | } - | ] - | }, - | "Properties": { - | "France": "Paris", - | "Germany": "Berlin", - | "Russia": "Moscow", - | "Ukraine": "Kiev" - | } - |} - """.stripMargin + """ + |{ + | "Event": "SparkListenerStageSubmitted", + | "Stage Info": { + | "Stage ID": 100, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 200, + | "RDD Info": [], + | "Parent IDs" : [100, 200, 300], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false + | }, + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1", + | "Internal": false, + | "Count Failed Values": false + | } + | ] + | }, + | "Properties": { + | "France": "Paris", + | "Germany": "Berlin", + | "Russia": "Moscow", + | "Ukraine": "Kiev" + | } + |} + """.stripMargin private val stageCompletedJsonString = """ @@ -1836,255 +1833,257 @@ private[spark] object JsonProtocolSuite extends Assertions { private val executorAddedJsonString = s""" - |{ - | "Event": "SparkListenerExecutorAdded", - | "Timestamp": ${executorAddedTime}, - | "Executor ID": "exec1", - | "Executor Info": { - | "Host": "Hostee.awesome.com", - | "Total Cores": 11, - | "Log Urls" : { - | "stderr" : "mystderr", - | "stdout" : "mystdout" - | } - | } - |} + |{ + | "Event": "SparkListenerExecutorAdded", + | "Timestamp": ${executorAddedTime}, + | "Executor ID": "exec1", + | "Executor Info": { + | "Host": "Hostee.awesome.com", + | "Total Cores": 11, + | "Log Urls" : { + | "stderr" : "mystderr", + | "stdout" : "mystdout" + | } + | } + |} """.stripMargin private val executorRemovedJsonString = s""" - |{ - | "Event": "SparkListenerExecutorRemoved", - | "Timestamp": ${executorRemovedTime}, - | "Executor ID": "exec2", - | "Removed Reason": "test reason" - |} + |{ + | "Event": "SparkListenerExecutorRemoved", + | "Timestamp": ${executorRemovedTime}, + | "Executor ID": "exec2", + | "Removed Reason": "test reason" + |} """.stripMargin private val executorMetricsUpdateJsonString = s""" - |{ - | "Event": "SparkListenerExecutorMetricsUpdate", - | "Executor ID": "exec3", - | "Metrics Updated": [ - | { - | "Task ID": 1, - | "Stage ID": 2, - | "Stage Attempt ID": 3, - | "Accumulator Updates": [ - | { - | "ID": 0, - | "Name": "$EXECUTOR_DESERIALIZE_TIME", - | "Update": 300, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 1, - | "Name": "$EXECUTOR_DESERIALIZE_CPU_TIME", - | "Update": 300, - | "Internal": true, - | "Count Failed Values": true - | }, - | - | { - | "ID": 2, - | "Name": "$EXECUTOR_RUN_TIME", - | "Update": 400, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 3, - | "Name": "$EXECUTOR_CPU_TIME", - | "Update": 400, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 4, - | "Name": "$RESULT_SIZE", - | "Update": 500, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 5, - | "Name": "$JVM_GC_TIME", - | "Update": 600, - | "Internal": true, - | "Count Failed Values": true - | }, + |{ + | "Event": "SparkListenerExecutorMetricsUpdate", + | "Executor ID": "exec3", + | "Metrics Updated": [ + | { + | "Task ID": 1, + | "Stage ID": 2, + | "Stage Attempt ID": 3, + | "Accumulator Updates": [ + | { + | "ID": 0, + | "Name": "$EXECUTOR_DESERIALIZE_TIME", + | "Update": 300, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 1, + | "Name": "$EXECUTOR_DESERIALIZE_CPU_TIME", + | "Update": 300, + | "Internal": true, + | "Count Failed Values": true + | }, + | | { - | "ID": 6, - | "Name": "$RESULT_SERIALIZATION_TIME", - | "Update": 700, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 7, - | "Name": "$MEMORY_BYTES_SPILLED", - | "Update": 800, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 8, - | "Name": "$DISK_BYTES_SPILLED", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 9, - | "Name": "$PEAK_EXECUTION_MEMORY", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 10, - | "Name": "$UPDATED_BLOCK_STATUSES", - | "Update": [ - | { - | "Block ID": "rdd_0_0", - | "Status": { - | "Storage Level": { - | "Use Disk": true, - | "Use Memory": true, - | "Deserialized": false, - | "Replication": 2 - | }, - | "Memory Size": 0, - | "Disk Size": 0 - | } - | } - | ], - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 11, - | "Name": "${shuffleRead.REMOTE_BLOCKS_FETCHED}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 12, - | "Name": "${shuffleRead.LOCAL_BLOCKS_FETCHED}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 13, - | "Name": "${shuffleRead.REMOTE_BYTES_READ}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 14, - | "Name": "${shuffleRead.REMOTE_BYTES_READ_TO_DISK}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 15, - | "Name": "${shuffleRead.LOCAL_BYTES_READ}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 16, - | "Name": "${shuffleRead.FETCH_WAIT_TIME}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 17, - | "Name": "${shuffleRead.RECORDS_READ}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 18, - | "Name": "${shuffleWrite.BYTES_WRITTEN}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 19, - | "Name": "${shuffleWrite.RECORDS_WRITTEN}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 20, - | "Name": "${shuffleWrite.WRITE_TIME}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 21, - | "Name": "${input.BYTES_READ}", - | "Update": 2100, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 22, - | "Name": "${input.RECORDS_READ}", - | "Update": 21, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 23, - | "Name": "${output.BYTES_WRITTEN}", - | "Update": 1200, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 24, - | "Name": "${output.RECORDS_WRITTEN}", - | "Update": 12, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 25, - | "Name": "$TEST_ACCUM", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | } - | ] - | } - | ], - | "Executor Metrics Updated" : { - | "JVMHeapMemory" : 543, - | "JVMOffHeapMemory" : 123456, - | "OnHeapExecutionMemory" : 12345, - | "OffHeapExecutionMemory" : 1234, - | "OnHeapStorageMemory" : 123, - | "OffHeapStorageMemory" : 12, - | "OnHeapUnifiedMemory" : 432, - | "OffHeapUnifiedMemory" : 321, - | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765 - | } - | + | "ID": 2, + | "Name": "$EXECUTOR_RUN_TIME", + | "Update": 400, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 3, + | "Name": "$EXECUTOR_CPU_TIME", + | "Update": 400, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 4, + | "Name": "$RESULT_SIZE", + | "Update": 500, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 5, + | "Name": "$JVM_GC_TIME", + | "Update": 600, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 6, + | "Name": "$RESULT_SERIALIZATION_TIME", + | "Update": 700, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 7, + | "Name": "$MEMORY_BYTES_SPILLED", + | "Update": 800, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 8, + | "Name": "$DISK_BYTES_SPILLED", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 9, + | "Name": "$PEAK_EXECUTION_MEMORY", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 10, + | "Name": "$UPDATED_BLOCK_STATUSES", + | "Update": [ + | { + | "Block ID": "rdd_0_0", + | "Status": { + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Deserialized": false, + | "Replication": 2 + | }, + | "Memory Size": 0, + | "Disk Size": 0 + | } + | } + | ], + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 11, + | "Name": "${shuffleRead.REMOTE_BLOCKS_FETCHED}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 12, + | "Name": "${shuffleRead.LOCAL_BLOCKS_FETCHED}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 13, + | "Name": "${shuffleRead.REMOTE_BYTES_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 14, + | "Name": "${shuffleRead.REMOTE_BYTES_READ_TO_DISK}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 15, + | "Name": "${shuffleRead.LOCAL_BYTES_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 16, + | "Name": "${shuffleRead.FETCH_WAIT_TIME}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 17, + | "Name": "${shuffleRead.RECORDS_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 18, + | "Name": "${shuffleWrite.BYTES_WRITTEN}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 19, + | "Name": "${shuffleWrite.RECORDS_WRITTEN}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 20, + | "Name": "${shuffleWrite.WRITE_TIME}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 21, + | "Name": "${input.BYTES_READ}", + | "Update": 2100, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 22, + | "Name": "${input.RECORDS_READ}", + | "Update": 21, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 23, + | "Name": "${output.BYTES_WRITTEN}", + | "Update": 1200, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 24, + | "Name": "${output.RECORDS_WRITTEN}", + | "Update": 12, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 25, + | "Name": "$TEST_ACCUM", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | } + | ] + | } + | ], + | "Executor Metrics Updated" : { + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "ProcessTreeRSSMemory": 123456, + | "ProcessTreeVMemory": 256912, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765 + | } + | |} """.stripMargin @@ -2098,6 +2097,8 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Executor Metrics" : { | "JVMHeapMemory" : 543, | "JVMOffHeapMemory" : 123456, + | "ProcessTreeRSSMemory": 123456, + | "ProcessTreeVMemory": 256912, | "OnHeapExecutionMemory" : 12345, | "OffHeapExecutionMemory" : 1234, | "OnHeapStorageMemory" : 123, @@ -2135,36 +2136,36 @@ private[spark] object JsonProtocolSuite extends Assertions { private val executorBlacklistedJsonString = s""" - |{ - | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorBlacklisted", - | "time" : ${executorBlacklistedTime}, - | "executorId" : "exec1", - | "taskFailures" : 22 - |} + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorBlacklisted", + | "time" : ${executorBlacklistedTime}, + | "executorId" : "exec1", + | "taskFailures" : 22 + |} """.stripMargin private val executorUnblacklistedJsonString = s""" - |{ - | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted", - | "time" : ${executorUnblacklistedTime}, - | "executorId" : "exec1" - |} + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted", + | "time" : ${executorUnblacklistedTime}, + | "executorId" : "exec1" + |} """.stripMargin private val nodeBlacklistedJsonString = s""" - |{ - | "Event" : "org.apache.spark.scheduler.SparkListenerNodeBlacklisted", - | "time" : ${nodeBlacklistedTime}, - | "hostId" : "node1", - | "executorFailures" : 33 - |} + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeBlacklisted", + | "time" : ${nodeBlacklistedTime}, + | "hostId" : "node1", + | "executorFailures" : 33 + |} """.stripMargin private val nodeUnblacklistedJsonString = s""" - |{ - | "Event" : "org.apache.spark.scheduler.SparkListenerNodeUnblacklisted", - | "time" : ${nodeUnblacklistedTime}, - | "hostId" : "node1" - |} + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeUnblacklisted", + | "time" : ${nodeUnblacklistedTime}, + | "hostId" : "node1" + |} """.stripMargin } From a0eed1187e93e80803af9ee4c2010c6d90cf1d31 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Sun, 5 Aug 2018 06:24:35 -0700 Subject: [PATCH 23/29] address code review comments --- .../scheduler/EventLoggingListener.scala | 8 ++--- .../spark/status/AppStatusListener.scala | 11 +++---- .../org/apache/spark/status/LiveEntity.scala | 2 +- .../org/apache/spark/status/api/v1/api.scala | 20 ++++-------- .../org/apache/spark/util/JsonProtocol.scala | 5 ++- .../scheduler/EventLoggingListenerSuite.scala | 32 +++++++++---------- .../spark/scheduler/ReplayListenerSuite.scala | 5 +-- .../spark/status/AppStatusListenerSuite.scala | 2 +- 8 files changed, 36 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index dd33505fc687..1629e1797977 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets import java.util.EnumSet import java.util.Locale -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.{ArrayBuffer, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} @@ -96,7 +96,7 @@ private[spark] class EventLoggingListener( private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) // map of (stageId, stageAttempt), to peak executor metrics for the stage - private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, ExecutorMetrics]]() + private val liveStageExecutorMetrics = Map.empty[(Int, Int), Map[String, ExecutorMetrics]] /** * Creates the log file in the configured log directory. @@ -165,7 +165,7 @@ private[spark] class EventLoggingListener( if (shouldLogStageExecutorMetrics) { // record the peak metrics for the new stage liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()), - new HashMap[String, ExecutorMetrics]()) + Map.empty[String, ExecutorMetrics]) } } @@ -338,7 +338,7 @@ private[spark] object EventLoggingListener extends Logging { private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) // A cache for compression codecs to avoid creating the same codec many times - private val codecMap = new HashMap[String, CompressionCodec] + private val codecMap = Map.empty[String, CompressionCodec] /** * Write metadata about an event log to the given stream. 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 836cef905aa8..a0f0c4ae4d59 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -706,13 +706,10 @@ private[spark] class AppStatusListener( // while reading from the log. SparkListenerStageExecutorMetrics are only processed // when reading logs. liveExecutors.get(executorMetrics.execId) - .orElse(deadExecutors.get(executorMetrics.execId)) match { - case Some(exec) => - if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) { - maybeUpdate(exec, now) - } - case None => - logWarning("unable to find executor " + executorMetrics.execId) + .orElse(deadExecutors.get(executorMetrics.execId)).map { exec => + if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) { + update(exec, now) + } } } 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 9c1ceb3a2559..a0b2458549fb 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -306,7 +306,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE executorLogs, memoryMetrics, blacklistedInStages, - if (peakExecutorMetrics.isSet()) Some(peakExecutorMetrics) else None) + Some(peakExecutorMetrics).filter(_.isSet)) 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 220be3ab141c..77466b62ff6e 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 @@ -116,33 +116,27 @@ class MemoryMetrics private[spark]( /** deserializer for peakMemoryMetrics: convert map to ExecutorMetrics */ private[spark] class ExecutorMetricsJsonDeserializer - extends JsonDeserializer[Option[ExecutorMetrics]] { + extends JsonDeserializer[Option[ExecutorMetrics]] { override def deserialize( jsonParser: JsonParser, deserializationContext: DeserializationContext): Option[ExecutorMetrics] = { val metricsMap = jsonParser.readValueAs[Option[Map[String, Long]]]( new TypeReference[Option[Map[String, java.lang.Long]]] {}) - metricsMap match { - case Some(metrics) => - Some(new ExecutorMetrics(metrics)) - case None => None - } + metricsMap.map(metrics => new ExecutorMetrics(metrics)) } } /** serializer for peakMemoryMetrics: convert ExecutorMetrics to map with metric name as key */ private[spark] class ExecutorMetricsJsonSerializer - extends JsonSerializer[Option[ExecutorMetrics]] { + extends JsonSerializer[Option[ExecutorMetrics]] { override def serialize( metrics: Option[ExecutorMetrics], jsonGenerator: JsonGenerator, serializerProvider: SerializerProvider): Unit = { - metrics match { - case Some(m) => - val metricsMap = ExecutorMetricType.values.map { metricType => + metrics.foreach { m: ExecutorMetrics => + val metricsMap = ExecutorMetricType.values.map { metricType => metricType.name -> m.getMetricValue(metricType) - }.toMap - jsonGenerator.writeObject(metricsMap) - case None => + }.toMap + jsonGenerator.writeObject(metricsMap) } } } 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 16613c3cc1e4..0cd8612b8fd1 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -723,9 +723,8 @@ private[spark] object JsonProtocol { (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) (taskId, stageId, stageAttemptId, updates) } - val executorUpdates = jsonOption(json \ "Executor Metrics Updated") match { - case None => None - case Some(executorUpdate) => Some(executorMetricsFromJson(executorUpdate)) + val executorUpdates = jsonOption(json \ "Executor Metrics Updated").map { + executorUpdate => executorMetricsFromJson(executorUpdate) } SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates) } 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 98cd7be53820..10ced5b27b6a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -272,22 +272,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus(conf) - // expected StageExecutorMetrics, for the given stage id and executor id - val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] = - Map( - ((0, "1"), - new SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), - ((0, "2"), - new SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), - ((1, "1"), - new SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), - ((1, "2"), - new SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) - // Events to post. val events = Array( SparkListenerApplicationStart("executionMetrics", None, @@ -356,6 +340,22 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit listenerBus.stop() eventLogger.stop() + // expected StageExecutorMetrics, for the given stage id and executor id + val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] = + Map( + ((0, "1"), + new SparkListenerStageExecutorMetrics("1", 0, 0, + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), + ((0, "2"), + new SparkListenerStageExecutorMetrics("2", 0, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), + ((1, "1"), + new SparkListenerStageExecutorMetrics("1", 1, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), + ((1, "2"), + new SparkListenerStageExecutorMetrics("2", 1, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) + // Verify the log file contains the expected events. // Posted events should be logged, except for ExecutorMetricsUpdate events -- these // are consolidated, and the peak values for each stage are logged at stage end. diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index f3eb986ff50f..d1113c7e0b10 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -219,10 +219,7 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp // Verify the same events are replayed in the same order assert(sc.eventLogger.isDefined) val originalEvents = sc.eventLogger.get.loggedEvents.filter { e => - JsonProtocol.sparkEventFromJson(e) match { - case event: SparkListenerStageExecutorMetrics => false - case _ => true - } + !JsonProtocol.sparkEventFromJson(e).isInstanceOf[SparkListenerStageExecutorMetrics] } val replayedEvents = eventMonster.loggedEvents originalEvents.zip(replayedEvents).foreach { case (e1, e2) => 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 f352ff6e457a..c78b7e69497c 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1291,7 +1291,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { test("stage executor metrics") { // simulate reading in StageExecutorMetrics events from the history log - val listener = new AppStatusListener(store, conf, true) + val listener = new AppStatusListener(store, conf, false) val driver = BlockManagerId(SparkContext.DRIVER_IDENTIFIER, "localhost", 42) listener.onExecutorAdded(createExecutorAddedEvent(1)) From 29a44c75309d13d071af29d5968f767fba7003d6 Mon Sep 17 00:00:00 2001 From: Reza Safi Date: Tue, 7 Aug 2018 12:03:46 -0400 Subject: [PATCH 24/29] Changing the position of ptree and also make the computation configurable and some improvments --- .../spark/executor/ProcfsBasedSystems.scala | 57 +++++++------------ .../apache/spark/memory/MemoryManager.scala | 7 --- .../spark/metrics/ExecutorMetricType.scala | 7 ++- 3 files changed, 26 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala index 1be94a644f48..09c034a2afd9 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala @@ -37,6 +37,8 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { val ptree: scala.collection.mutable.Map[ Int, Set[Int]] = scala.collection.mutable.Map[ Int, Set[Int]]() val PROCFS_STAT_FILE = "stat" + var latestVmemTotal: Long = 0 + var latestRSSTotal: Long = 0 createProcessTree @@ -53,7 +55,10 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { catch { case f: FileNotFoundException => return false } - true + + val shouldLogStageExecutorProcessTreeMetrics = org.apache.spark.SparkEnv.get.conf. + getBoolean("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled", true) + true && shouldLogStageExecutorProcessTreeMetrics } @@ -62,6 +67,8 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { return -1; } try { + // This can be simplified in java9: + // https://docs.oracle.com/javase/9/docs/api/java/lang/ProcessHandle.html val cmd = Array("bash", "-c", "echo $PPID") val length = 10 var out: Array[Byte] = Array.fill[Byte](length)(0) @@ -136,7 +143,7 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { * instead. The computation of RSS and Vmem are based on proc(5): * http://man7.org/linux/man-pages/man5/proc.5.html */ - def getProcessInfo(pid: Int): String = { + def getProcessInfo(pid: Int): Unit = { try { val pidDir: File = new File(procfsDir, pid.toString) val fReader = new InputStreamReader( @@ -146,11 +153,14 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { val procInfo = in.readLine in.close fReader.close - return procInfo + val procInfoSplit = procInfo.split(" ") + if ( procInfoSplit != null ) { + latestVmemTotal += procInfoSplit(22).toLong + latestRSSTotal += procInfoSplit(23).toLong + } } catch { case f: FileNotFoundException => return null } - null } @@ -160,46 +170,22 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { } updateProcessTree val pids = ptree.keySet - var totalRss = 0L + latestRSSTotal = 0 + latestVmemTotal = 0 for (p <- pids) { - totalRss += getProcessRSSInfo(p) + getProcessInfo(p) } - totalRss + latestRSSTotal } - def getProcessRSSInfo(pid: Int): Long = { - val pInfo = getProcessInfo(pid) - if (pInfo != null) { - val pInfoSplit = pInfo.split(" ") - // According to proc(5) RSS is the 24th value when we read first line of /proc/[pid]/stat - return pInfoSplit(23).toLong - } - 0 - } def getVirtualMemInfo(): Long = { if (!isAvailable) { return -1 } - // We won't call updateProcessTree here since we already did that when we - // computed RSS info - val pids = ptree.keySet - var totalVMem = 0L - for (p <- pids) { - totalVMem += getProcessVirtualMemInfo(p) - } - totalVMem - } - - - def getProcessVirtualMemInfo(pid: Int): Long = { - val pInfo = getProcessInfo(pid) - if (pInfo != null) { - val pInfoSplit = pInfo.split(" ") - // According to proc(5) Vmem is the 23rd value when we read first line of /proc/[pid]/stat - return pInfoSplit(22).toLong - } - 0L + // We won't call updateProcessTree and also compute total virtual memory here + // since we already did all of this when we computed RSS info + latestVmemTotal } @@ -233,5 +219,4 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { return new mutable.ArrayBuffer() } } - } diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 92a59b20417c..965515b8abe1 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -20,7 +20,6 @@ package org.apache.spark.memory import javax.annotation.concurrent.GuardedBy import org.apache.spark.SparkConf -import org.apache.spark.executor.{ProcessTreeMetrics, ProcfsBasedSystems} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.storage.BlockId @@ -201,12 +200,6 @@ private[spark] abstract class MemoryManager( */ final def offHeapStorageMemoryUsed: Long = offHeapStorageMemoryPool.memoryUsed - /** - * If the system isn't procfsBased the process tree metrics' values will be -1, - * meaning not available - */ - final val pTreeInfo: ProcessTreeMetrics = new ProcfsBasedSystems - /** * Returns the execution memory consumption, in bytes, for the given task. */ diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index 23c2d18359bf..fb52fac1d346 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -19,6 +19,7 @@ package org.apache.spark.metrics import java.lang.management.{BufferPoolMXBean, ManagementFactory} import javax.management.ObjectName +import org.apache.spark.executor.{ProcessTreeMetrics, ProcfsBasedSystems} import org.apache.spark.memory.MemoryManager /** @@ -61,13 +62,13 @@ case object JVMOffHeapMemory extends ExecutorMetricType { case object ProcessTreeRSSMemory extends ExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { - memoryManager.pTreeInfo.getRSSInfo() + ExecutorMetricType.pTreeInfo.getRSSInfo() } } case object ProcessTreeVMemory extends ExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { - memoryManager.pTreeInfo.getVirtualMemInfo() + ExecutorMetricType.pTreeInfo.getVirtualMemInfo() } } @@ -96,6 +97,8 @@ case object MappedPoolMemory extends MBeanExecutorMetricType( "java.nio:type=BufferPool,name=mapped") private[spark] object ExecutorMetricType { + final val pTreeInfo: ProcessTreeMetrics = new ProcfsBasedSystems + // List of all executor metric types val values = IndexedSeq( JVMHeapMemory, From 36714270d57e3151472363ff2e2c14ab6b88bdea Mon Sep 17 00:00:00 2001 From: Reza Safi Date: Wed, 8 Aug 2018 17:05:50 -0400 Subject: [PATCH 25/29] Seperate metrics for jvm, python and others and update the tests --- .../spark/executor/ProcessTreeMetrics.scala | 8 +- .../spark/executor/ProcfsBasedSystems.scala | 69 +- .../spark/metrics/ExecutorMetricType.scala | 42 +- ...ith_executor_metrics_json_expectation.json | 50 +- .../application_1506645932520_24630151 | 20 +- .../scheduler/EventLoggingListenerSuite.scala | 85 ++- .../spark/status/AppStatusListenerSuite.scala | 93 ++- .../apache/spark/util/JsonProtocolSuite.scala | 675 +++++++++--------- 8 files changed, 589 insertions(+), 453 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ProcessTreeMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ProcessTreeMetrics.scala index 87d1356890e6..100ba6c282ce 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcessTreeMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcessTreeMetrics.scala @@ -23,6 +23,10 @@ private[spark] trait ProcessTreeMetrics { def computePid(): Int def createProcessTree() def updateProcessTree() - def getRSSInfo(): Long - def getVirtualMemInfo(): Long + def getJVMRSSInfo(): Long + def getJVMVirtualMemInfo(): Long + def getPythonRSSInfo(): Long + def getPythonVirtualMemInfo(): Long + def getOtherRSSInfo(): Long + def getOtherVirtualMemInfo(): Long } diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala index 09c034a2afd9..91420dab3b5a 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsBasedSystems.scala @@ -37,8 +37,12 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { val ptree: scala.collection.mutable.Map[ Int, Set[Int]] = scala.collection.mutable.Map[ Int, Set[Int]]() val PROCFS_STAT_FILE = "stat" - var latestVmemTotal: Long = 0 - var latestRSSTotal: Long = 0 + var latestJVMVmemTotal: Long = 0 + var latestJVMRSSTotal: Long = 0 + var latestPythonVmemTotal: Long = 0 + var latestPythonRSSTotal: Long = 0 + var latestOtherVmemTotal: Long = 0 + var latestOtherRSSTotal: Long = 0 createProcessTree @@ -155,8 +159,17 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { fReader.close val procInfoSplit = procInfo.split(" ") if ( procInfoSplit != null ) { - latestVmemTotal += procInfoSplit(22).toLong - latestRSSTotal += procInfoSplit(23).toLong + if (procInfoSplit(1).toLowerCase.contains("java")) { + latestJVMVmemTotal += procInfoSplit(22).toLong + latestJVMRSSTotal += procInfoSplit(23).toLong + } + else if (procInfoSplit(1).toLowerCase.contains("python")) { + latestPythonVmemTotal += procInfoSplit(22).toLong + latestPythonRSSTotal += procInfoSplit(23).toLong + } + else { + latestOtherVmemTotal += procInfoSplit(22).toLong + latestOtherRSSTotal += procInfoSplit(23).toLong } } } catch { case f: FileNotFoundException => return null @@ -164,28 +177,64 @@ class ProcfsBasedSystems extends ProcessTreeMetrics with Logging { } - def getRSSInfo(): Long = { + def getOtherRSSInfo(): Long = { if (!isAvailable) { return -1 } updateProcessTree val pids = ptree.keySet - latestRSSTotal = 0 - latestVmemTotal = 0 + latestJVMRSSTotal = 0 + latestJVMVmemTotal = 0 + latestPythonRSSTotal = 0 + latestPythonVmemTotal = 0 + latestOtherRSSTotal = 0 + latestOtherVmemTotal = 0 for (p <- pids) { getProcessInfo(p) } - latestRSSTotal + latestOtherRSSTotal } - def getVirtualMemInfo(): Long = { + def getOtherVirtualMemInfo(): Long = { if (!isAvailable) { return -1 } // We won't call updateProcessTree and also compute total virtual memory here // since we already did all of this when we computed RSS info - latestVmemTotal + latestOtherVmemTotal + } + + + def getJVMRSSInfo(): Long = { + if (!isAvailable) { + return -1 + } + latestJVMRSSTotal + } + + + def getJVMVirtualMemInfo(): Long = { + if (!isAvailable) { + return -1 + } + latestJVMVmemTotal + } + + + def getPythonRSSInfo(): Long = { + if (!isAvailable) { + return -1 + } + latestPythonRSSTotal + } + + + def getPythonVirtualMemInfo(): Long = { + if (!isAvailable) { + return -1 + } + latestPythonVmemTotal } diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index fb52fac1d346..efc38af8800b 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -60,15 +60,39 @@ case object JVMOffHeapMemory extends ExecutorMetricType { } } -case object ProcessTreeRSSMemory extends ExecutorMetricType { +case object ProcessTreeJVMRSSMemory extends ExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { - ExecutorMetricType.pTreeInfo.getRSSInfo() + ExecutorMetricType.pTreeInfo.getJVMRSSInfo() } } -case object ProcessTreeVMemory extends ExecutorMetricType { +case object ProcessTreeJVMVMemory extends ExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { - ExecutorMetricType.pTreeInfo.getVirtualMemInfo() + ExecutorMetricType.pTreeInfo.getJVMVirtualMemInfo() + } +} + +case object ProcessTreePythonRSSMemory extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { + ExecutorMetricType.pTreeInfo.getPythonRSSInfo() + } +} + +case object ProcessTreePythonVMemory extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { + ExecutorMetricType.pTreeInfo.getPythonVirtualMemInfo() + } +} + +case object ProcessTreeOtherRSSMemory extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { + ExecutorMetricType.pTreeInfo.getOtherRSSInfo() + } +} + +case object ProcessTreeOtherVMemory extends ExecutorMetricType { + override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { + ExecutorMetricType.pTreeInfo.getOtherVirtualMemInfo() } } @@ -103,8 +127,6 @@ private[spark] object ExecutorMetricType { val values = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, - ProcessTreeRSSMemory, - ProcessTreeVMemory, OnHeapExecutionMemory, OffHeapExecutionMemory, OnHeapStorageMemory, @@ -112,7 +134,13 @@ private[spark] object ExecutorMetricType { OnHeapUnifiedMemory, OffHeapUnifiedMemory, DirectPoolMemory, - MappedPoolMemory + MappedPoolMemory, + ProcessTreeJVMVMemory, + ProcessTreeJVMRSSMemory, + ProcessTreePythonVMemory, + ProcessTreePythonRSSMemory, + ProcessTreeOtherVMemory, + ProcessTreeOtherRSSMemory ) // Map of executor metric type to its index in values. 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 03735714db05..bdee36a5a97e 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 @@ -30,8 +30,6 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 905801, "JVMOffHeapMemory" : 205304696, - "ProcessTreeRSSMemory": 200000000, - "ProcessTreeVMemory": 500000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 905801, "OnHeapExecutionMemory" : 0, @@ -39,7 +37,13 @@ "DirectPoolMemory" : 397602, "MappedPoolMemory" : 0, "JVMHeapMemory" : 629553808, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 500000000, + "ProcessTreeJVMRSSMemory": 200000000, + "ProcessTreePythonVMemory": 80000000, + "ProcessTreePythonRSSMemory": 20000000, + "ProcessTreeOtherVMemory": 80000000, + "ProcessTreeOtherRSSMemory": 20000000 } }, { "id" : "7", @@ -172,8 +176,6 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 63104457, "JVMOffHeapMemory" : 95657456, - "ProcessTreeRSSMemory": 100000000, - "ProcessTreeVMemory": 500000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 100853193, "OnHeapExecutionMemory" : 37748736, @@ -181,7 +183,13 @@ "DirectPoolMemory" : 126261, "MappedPoolMemory" : 0, "JVMHeapMemory" : 518613056, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 500000000, + "ProcessTreeJVMRSSMemory": 100000000, + "ProcessTreePythonVMemory": 80000000, + "ProcessTreePythonRSSMemory": 20000000, + "ProcessTreeOtherVMemory": 80000000, + "ProcessTreeOtherRSSMemory": 20000000 } }, { "id" : "3", @@ -218,8 +226,6 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 69535048, "JVMOffHeapMemory" : 90709624, - "ProcessTreeRSSMemory": 200000000, - "ProcessTreeVMemory": 600000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 69535048, "OnHeapExecutionMemory" : 0, @@ -227,7 +233,13 @@ "DirectPoolMemory" : 87796, "MappedPoolMemory" : 0, "JVMHeapMemory" : 726805712, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 600000000, + "ProcessTreeJVMRSSMemory": 200000000, + "ProcessTreePythonVMemory": 200000000, + "ProcessTreePythonRSSMemory": 70000000, + "ProcessTreeOtherVMemory": 200000000, + "ProcessTreeOtherRSSMemory": 70000000 } }, { "id" : "2", @@ -264,8 +276,6 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 58468944, "JVMOffHeapMemory" : 91208368, - "ProcessTreeRSSMemory": 90000000, - "ProcessTreeVMemory": 400000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 58468944, "OnHeapExecutionMemory" : 0, @@ -273,7 +283,13 @@ "DirectPoolMemory" : 87796, "MappedPoolMemory" : 0, "JVMHeapMemory" : 595946552, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 400000000, + "ProcessTreeJVMRSSMemory": 90000000, + "ProcessTreePythonVMemory": 100000000, + "ProcessTreePythonRSSMemory": 8000000, + "ProcessTreeOtherVMemory": 100000000, + "ProcessTreeOtherRSSMemory": 8000000 } }, { "id" : "1", @@ -310,8 +326,6 @@ "peakMemoryMetrics" : { "OnHeapStorageMemory" : 47962185, "JVMOffHeapMemory" : 100519936, - "ProcessTreeRSSMemory": 100000000, - "ProcessTreeVMemory": 500000000, "OffHeapExecutionMemory" : 0, "OnHeapUnifiedMemory" : 47962185, "OnHeapExecutionMemory" : 0, @@ -319,6 +333,12 @@ "DirectPoolMemory" : 98230, "MappedPoolMemory" : 0, "JVMHeapMemory" : 755008624, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 500000000, + "ProcessTreeJVMRSSMemory": 100000000, + "ProcessTreePythonVMemory": 400000000, + "ProcessTreePythonRSSMemory": 40000000, + "ProcessTreeOtherVMemory": 400000000, + "ProcessTreeOtherRSSMemory": 40000000 } } ] diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151 index b6d9b759a920..ce28c43bb909 100644 --- a/core/src/test/resources/spark-events/application_1506645932520_24630151 +++ b/core/src/test/resources/spark-events/application_1506645932520_24630151 @@ -33,11 +33,11 @@ {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1524182142251,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144246,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1920975","Value":"1920974","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3562","Value":"3562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"41943038","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"38","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1813","Value":"1812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"195602","Value":"195602","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3563","Value":"3563","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1558","Value":"1557","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3563,"Value":3563,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36845111,"Value":36845111,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":27318908,"Value":27318908,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3562,"Value":3562,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349287,"Value":349287,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":33,"Value":33,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1498974375,"Value":1498974375,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1922,"Value":1922,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":49547405,"Value":49547405,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":56,"Value":56,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":56,"Executor Deserialize CPU Time":49547405,"Executor Run Time":1922,"Executor CPU Time":1498974375,"Result Size":2394,"JVM GC Time":33,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349287,"Shuffle Write Time":27318908,"Shuffle Records Written":3562},"Input Metrics":{"Bytes Read":36845111,"Records Read":3563},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1524182130331,"Executor ID":"2","Host":"node4045.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144444,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"204058975","Value":"564814764","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"616897","Value":"1707779","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"23365","Value":"63634","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50423423,"Value":138656729,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":105575962,"Value":301246724,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":616897,"Value":1707779,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":22950296,"Value":69377072,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":326,"Value":1107,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":5568,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":11931694025,"Value":31991331624,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13454,"Value":36578,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":531799977,"Value":1633060096,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":594,"Value":1930,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":594,"Executor Deserialize CPU Time":531799977,"Executor Run Time":13454,"Executor CPU Time":11931694025,"Result Size":1856,"JVM GC Time":326,"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":22950296,"Shuffle Write Time":105575962,"Shuffle Records Written":616897},"Input Metrics":{"Bytes Read":50423423,"Records Read":616897},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1524182130349,"Executor ID":"3","Host":"node0998.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182144840,"Failed":false,"Killed":false,"Accumulables":[{"ID":7,"Name":"data size total (min, med, max)","Update":"207338935","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"626277","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":5,"Name":"duration total (min, med, max)","Update":"24254","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":59,"Name":"internal.metrics.input.recordsRead","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Update":50409514,"Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Update":106963069,"Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":626277,"Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":31362123,"Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Update":342,"Value":1449,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.resultSize","Update":1856,"Value":7424,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Update":12267596062,"Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Update":13858,"Value":50436,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Update":519573839,"Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Update":573,"Value":2503,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":573,"Executor Deserialize CPU Time":519573839,"Executor Run Time":13858,"Executor CPU Time":12267596062,"Result Size":1856,"JVM GC Time":342,"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":31362123,"Shuffle Write Time":106963069,"Shuffle Records Written":626277},"Input Metrics":{"Bytes Read":50409514,"Records Read":626277},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":592412824,"JVMOffHeapMemory":202907152,"ProcessTreeRSSMemory":90000000,"ProcessTreeVMemory":500000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":355389,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":523121272,"JVMOffHeapMemory":88280720,"ProcessTreeRSSMemory":90000000,"ProcessTreeVMemory":400000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":52050147,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":52050147,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":214174608,"JVMOffHeapMemory":91548704,"ProcessTreeRSSMemory":85000000,"ProcessTreeVMemory":450000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47399168,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47399168,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"ProcessTreeRSSMemory":95000000,"ProcessTreeVMemory":480000000,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"ProcessTreeRSSMemory":200000000,"ProcessTreeVMemory":600000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":592412824,"JVMOffHeapMemory":202907152,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":355389,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":500000000,"ProcessTreeJVMRSSMemory":90000000,"ProcessTreePythonVMemory":70000000,"ProcessTreePythonRSSMemory":10000000,"ProcessTreeOtherVMemory":70000000,"ProcessTreeOtherRSSMemory":10000000}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":523121272,"JVMOffHeapMemory":88280720,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":52050147,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":52050147,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":400000000,"ProcessTreeJVMRSSMemory":90000000,"ProcessTreePythonVMemory":100000000,"ProcessTreePythonRSSMemory":8000000,"ProcessTreeOtherVMemory":100000000,"ProcessTreeOtherRSSMemory":8000000}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":214174608,"JVMOffHeapMemory":91548704,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47399168,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47399168,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":450000000,"ProcessTreeJVMRSSMemory":85000000,"ProcessTreePythonVMemory":300000000,"ProcessTreePythonRSSMemory":30000000,"ProcessTreeOtherVMemory":300000000,"ProcessTreeOtherRSSMemory":30000000}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":480000000,"ProcessTreeJVMRSSMemory":95000000,"ProcessTreePythonVMemory":80000000,"ProcessTreePythonRSSMemory":20000000,"ProcessTreeOtherVMemory":80000000,"ProcessTreeOtherRSSMemory":20000000}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":500000000,"ProcessTreeJVMRSSMemory":100000000,"ProcessTreePythonVMemory":100000000,"ProcessTreePythonRSSMemory":60000000,"ProcessTreeOtherVMemory":100000000,"ProcessTreeOtherRSSMemory":60000000}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130229,"Completion Time":1524182144852,"Accumulables":[{"ID":41,"Name":"internal.metrics.resultSize","Value":7424,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.input.recordsRead","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":38,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2152633935,"Internal":true,"Count Failed Values":true},{"ID":56,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2334056,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"duration total (min, med, max)","Value":"87888","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":55,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":100739195,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorCpuTime","Value":44258927686,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.input.bytesRead","Value":189066243,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"data size total (min, med, max)","Value":"772153699","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"2334056","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":37,"Name":"internal.metrics.executorDeserializeTime","Value":2503,"Internal":true,"Count Failed Values":true},{"ID":57,"Name":"internal.metrics.shuffle.write.writeTime","Value":408209793,"Internal":true,"Count Failed Values":true},{"ID":39,"Name":"internal.metrics.executorRunTime","Value":50436,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.jvmGCTime","Value":1449,"Internal":true,"Count Failed Values":true}]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1524182142997,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145327,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1953295","Value":"3874269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3575","Value":"7137","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"83886077","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"49","Value":"86","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"2002","Value":"3814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"196587","Value":"392189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3575","Value":"7138","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1755","Value":"3312","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3575,"Value":7138,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36849246,"Value":73694357,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":32035583,"Value":59354491,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3575,"Value":7137,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":349006,"Value":698293,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":83886080,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":4788,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":1785119941,"Value":3284094316,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":2182,"Value":4104,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":71500541,"Value":121047946,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":136,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":136,"Executor Deserialize CPU Time":71500541,"Executor Run Time":2182,"Executor CPU Time":1785119941,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":349006,"Shuffle Write Time":32035583,"Shuffle Records Written":3575},"Input Metrics":{"Bytes Read":36849246,"Records Read":3575},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1524182144237,"Executor ID":"1","Host":"node1404.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182145971,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1337999","Value":"5212268","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"2435","Value":"9572","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"37748735","Value":"121634812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"9","Value":"95","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"1703","Value":"5517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"133759","Value":"525948","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"2435","Value":"9573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"1609","Value":"4921","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":2435,"Value":9573,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":24250210,"Value":97944567,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":20055909,"Value":79410400,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2435,"Value":9572,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":242714,"Value":941007,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":37748736,"Value":121634816,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":31,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2394,"Value":7182,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":896878991,"Value":4180973307,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":1722,"Value":5826,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2787355,"Value":123835301,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":195,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2787355,"Executor Run Time":1722,"Executor CPU Time":896878991,"Result Size":2394,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":242714,"Shuffle Write Time":20055909,"Shuffle Records Written":2435},"Input Metrics":{"Bytes Read":24250210,"Records Read":2435},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} @@ -46,11 +46,11 @@ {"Event":"SparkListenerExecutorAdded","Timestamp":1524182149826,"Executor ID":"7","Executor Info":{"Host":"node6340.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096","stderr":"http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"node6340.grid.company.com","Port":5933},"Maximum Memory":956615884,"Timestamp":1524182149983,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} {"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":2,"Attempt":0,"Launch Time":1524182143166,"Executor ID":"5","Host":"node2477.grid.company.com","Locality":"ANY","Speculative":false,"Getting Result Time":0,"Finish Time":1524182152418,"Failed":false,"Killed":false,"Accumulables":[{"ID":8,"Name":"data size total (min, med, max)","Update":"1910103","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":23,"Name":"number of output rows","Update":"3541","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":25,"Name":"peak memory total (min, med, max)","Update":"41943039","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":24,"Name":"sort time total (min, med, max)","Update":"48","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"duration total (min, med, max)","Update":"6093","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Update":"194553","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":29,"Name":"number of output rows","Update":"3541","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Update":"5951","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":84,"Name":"internal.metrics.input.recordsRead","Update":3541,"Value":13114,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.input.bytesRead","Update":36838295,"Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Update":49790497,"Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3541,"Value":13113,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355051,"Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Update":41943040,"Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Update":920,"Value":1015,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Update":2437,"Value":9619,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorCpuTime","Update":5299274511,"Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Update":7847,"Value":13673,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Update":687811857,"Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Update":1037,"Value":1232,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1037,"Executor Deserialize CPU Time":687811857,"Executor Run Time":7847,"Executor CPU Time":5299274511,"Result Size":2437,"JVM GC Time":920,"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":355051,"Shuffle Write Time":49790497,"Shuffle Records Written":3541},"Input Metrics":{"Bytes Read":36838295,"Records Read":3541},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":629553808,"JVMOffHeapMemory":205304696,"ProcessTreeRSSMemory":200000000,"ProcessTreeVMemory":400000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":397602,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":595946552,"JVMOffHeapMemory":91208368,"ProcessTreeRSSMemory":80000000,"ProcessTreeVMemory":300000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":58468944,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":58468944,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":755008624,"JVMOffHeapMemory":100519936,"ProcessTreeRSSMemory":100000000,"ProcessTreeVMemory":500000000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47962185,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47962185,"OffHeapUnifiedMemory":0,"DirectPoolMemory":98230,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"ProcessTreeRSSMemory": 100000000,"ProcessTreeVMemory": 500000000,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":629553808,"JVMOffHeapMemory":205304696,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":905801,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":905801,"OffHeapUnifiedMemory":0,"DirectPoolMemory":397602,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":400000000,"ProcessTreeJVMRSSMemory":200000000,"ProcessTreePythonVMemory":80000000,"ProcessTreePythonRSSMemory":20000000,"ProcessTreeOtherVMemory":80000000,"ProcessTreeOtherRSSMemory":20000000}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":595946552,"JVMOffHeapMemory":91208368,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":58468944,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":58468944,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":300000000,"ProcessTreeJVMRSSMemory":80000000,"ProcessTreePythonVMemory":90000000,"ProcessTreePythonRSSMemory":7000000,"ProcessTreeOtherVMemory":90000000,"ProcessTreeOtherRSSMemory":7000000}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":755008624,"JVMOffHeapMemory":100519936,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":47962185,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":47962185,"OffHeapUnifiedMemory":0,"DirectPoolMemory":98230,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":500000000,"ProcessTreeJVMRSSMemory":100000000,"ProcessTreePythonVMemory":400000000,"ProcessTreePythonRSSMemory":40000000,"ProcessTreeOtherVMemory":400000000,"ProcessTreeOtherRSSMemory":40000000}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":518613056,"JVMOffHeapMemory":95657456,"OnHeapExecutionMemory":37748736,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":63104457,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":100853193,"OffHeapUnifiedMemory":0,"DirectPoolMemory":126261,"MappedPoolMemory":0,"ProcessTreeJVMVMemory": 500000000,"ProcessTreeJVMRSSMemory": 100000000,"ProcessTreePythonVMemory":60000000,"ProcessTreePythonRSSMemory":10000000,"ProcessTreeOtherVMemory":60000000,"ProcessTreeOtherRSSMemory":10000000}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":726805712,"JVMOffHeapMemory":90709624,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":69535048,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":69535048,"OffHeapUnifiedMemory":0,"DirectPoolMemory":87796,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":600000000,"ProcessTreeJVMRSSMemory":200000000,"ProcessTreePythonVMemory":200000000,"ProcessTreePythonRSSMemory":70000000,"ProcessTreeOtherVMemory":200000000,"ProcessTreeOtherRSSMemory":70000000}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182130328,"Completion Time":1524182152419,"Accumulables":[{"ID":83,"Name":"internal.metrics.input.bytesRead","Value":134782862,"Internal":true,"Count Failed Values":true},{"ID":23,"Name":"number of output rows","Value":"13113","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":68,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"data size total (min, med, max)","Value":"7122371","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":62,"Name":"internal.metrics.executorDeserializeTime","Value":1232,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":1296058,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.peakExecutionMemory","Value":163577856,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"number of output rows","Value":"13114","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":65,"Name":"internal.metrics.executorCpuTime","Value":9480247818,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.executorRunTime","Value":13673,"Internal":true,"Count Failed Values":true},{"ID":82,"Name":"internal.metrics.shuffle.write.writeTime","Value":129200897,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.jvmGCTime","Value":1015,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"peak memory total (min, med, max)","Value":"163577851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":28,"Name":"number of output rows","Value":"720501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":63,"Name":"internal.metrics.executorDeserializeCpuTime","Value":811647158,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"duration total (min, med, max)","Value":"11610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":13113,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.input.recordsRead","Value":13114,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.resultSize","Value":9619,"Internal":true,"Count Failed Values":true},{"ID":24,"Name":"sort time total (min, med, max)","Value":"143","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":33,"Name":"duration total (min, med, max)","Value":"10872","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"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":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","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":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"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\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","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":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"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":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","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},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"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":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"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":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"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":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"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":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"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":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"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":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Submission Time":1524182152430,"Accumulables":[]},"Properties":{"spark.sql.execution.id":"2"}} {"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1524182152447,"Executor ID":"4","Host":"node4243.grid.company.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} 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 51117e39a986..f1fb9468641c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -282,58 +282,67 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 3500L, 7500L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), + new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(1500L, 50L, 3500L, 8500L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), - // exec 1: new stage 0 peaks for metrics at indexes: 2, 3, 4, 6, 8 + new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))), + // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 4000L, 8000L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 2, 3, 6, 8 + new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(2000L, 50L, 4000L, 9000L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), - // exec 1: new stage 0 peaks for metrics at indexes: 7, 9 + new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L))), + // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(2000L, 40L, 3500L, 8000L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 7, 8, 9, 10 + new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(3500L, 50L, 3500L, 8500L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), + new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), - // exec 1: new stage 0 peaks for metrics at indexes: 0, 5, 9; initialize stage 1 peaks + // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5000L, 30L, 3000L, 5000L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, - 0L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 4, 5, 8, 9, 11; + new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, + 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 3, 6, 7, 9; // initialize stage 1 peaks createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(7000L, 70L, 4000L, 8000L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, - 40L))), + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, + 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L))), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), - // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 2, 3, 5 + // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(6000L, 70L, 3000L, 5000L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), - // exec 2: new stage 1 peaks for metrics at indexes: 2, 3, 5, 6, 9, 10 + new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))), + // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 30L, 5000L, 8000L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, - 20L))), - // exec 1: new stage 1 peaks for metrics at indexes: 0, 6, 7, 9 + new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, + 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 5000L, 2000L))), + // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(7000L, 70L, 2500L, 3000L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), - // exec 2: new stage 1 peak for metrics at index: 9 + new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, + 2000L, 1500L, 1000L, 500L))), + // exec 2: new stage 1 peak for metrics at index: 7 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 40L, 3000L, 7000L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, - 20L))), + new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, + 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L))), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5500L, 70L, 2500L, 4000L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, - 0L))), + new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, + 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L))), createExecutorRemovedEvent(1), - // exec 2: new stage 1 peak for metrics at index: 8 + // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(4000L, 20L, 4000L, 7000L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), + new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, + 4000L, 6000L, 3000L, 5000L, 2000L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -350,20 +359,20 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit Map( ((0, "1"), new SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 4000L, 8000L, 50L, 20L, 50L, 10L, 100L, 30L, - 70L, 20L)))), + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)))), ((0, "2"), new SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 4000L, 9000L, 50L, 20L, 10L, 10L, 50L, 30L, - 80L, 40L)))), + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), ((1, "1"), new SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 3000L, 5000L, 50L, 30L, 60L, 30L, 80L, 55L, - 50L, 0L)))), + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, + 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)))), ((1, "2"), new SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 5000L, 8000L, 50L, 40L, 10L, 30L, 50L, 60L, - 40L, 40L))))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, + 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) // Verify the log file contains the expected events. // Posted events should be logged, except for ExecutorMetricsUpdate events -- these 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 c5190ca7a71c..bfd9cda7d81c 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1220,60 +1220,74 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 3500L, 7500L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(1500L, 50L, 3500L, 8500L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) - // exec 1: new stage 0 peaks for metrics at indexes: 2, 3, 4, 6, 8 + Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))) + // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 4000L, 8000L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) - // exec 2: new stage 0 peaks for metrics at indexes: 0, 2, 3, 6, 8 + Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(2000L, 50L, 4000L, 9000L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) - // exec 1: new stage 0 peaks for metrics at indexes: 7, 9 + Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L))) + // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(2000L, 40L, 3500L, 8000L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) - // exec 2: new stage 0 peaks for metrics at indexes: 0, 7, 8, 9, 10 + Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(3500L, 50L, 3500L, 8500L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))) // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks listener.onStageSubmitted(createStageSubmittedEvent(1)) - // exec 1: new stage 0 peaks for metrics at indexes: 0, 5, 9 + // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5000L, 30L, 3000L, 5000L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) - // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 4, 5, 8, 9, 11 + Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))) + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(7000L, 80L, 4000L, 8000L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))) // complete stage 0, and 3 more updates for each executor with just // stage 1 running listener.onStageCompleted(createStageCompletedEvent(0)) - // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 5 + // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(6000L, 70L, 3000L, 5000L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) - // exec 2: new stage 1 peaks for metrics at indexes: 5, 6, 9, 10 + Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))) + // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 30L, 5000L, 8000L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) - // exec 1: new stage 1 peaks for metrics at indexes: 0, 6, 7, 9 + Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L, 8000L, 5000L, + 7000L, 4000L, 6000L, 3000L))) + // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(7000L, 70L, 2500L, 3000L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) - // exec 2: new stage 1 peak for metrics at index: 9 + Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, 2000L, + 1500L, 1000L, 500L))) + // exec 2: new stage 1 peak for metrics at index: 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 40L, 3000L, 7000L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L, 7000L, 3000L, + 6000L, 2000L, 5000L, 1000L))) // exec 1: no new stage 1 peaks listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5500L, 70L, 2500L, 4000L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L, 4000L, 2500L, + 3000L, 1500, 2000L, 500L))) listener.onExecutorRemoved(createExecutorRemovedEvent(1)) - // exec 2: new stage 1 peak for metrics at index: 8 + // exec 2: new stage 1 peak for metrics at index: 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(4000L, 20L, 4000L, 7000L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, 4000L, 6000L, + 3000L, 5000L, 2000L))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 4000L, 8000L, 50L, 30L, 60L, 30L, 100L, 55L, - 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 5000L, 9000L, 50L, 40L, 10L, 30L, 50L, 60L, - 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L, 9000L, 5000L, 8000L, 4000L, 7000L, 3000L))) // check that the stored peak values match the expected values expectedValues.foreach { case (id, metrics) => @@ -1301,26 +1315,29 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onStageSubmitted(createStageSubmittedEvent(0)) listener.onStageSubmitted(createStageSubmittedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 4000L, 8000L, 50L, 20L, 50L, 10L, 100L, 30L, - 70L, 20L)))) + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 4000L, 9000L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L, 9000L, + 4000L, 8000L, 3000L, 7000L, 2000L)))) listener.onStageCompleted(createStageCompletedEvent(0)) // executor 1 is removed before stage 1 has finished, the stage executor metrics // are logged afterwards and should still be used to update the executor metrics. listener.onExecutorRemoved(createExecutorRemovedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 3000L, 5000L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 80L, 5000L, 8000L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L, 8000L, 5000L, + 7000L, 4000L, 6000L, 3000L)))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 4000L, 8000L, 50L, 30L, 60L, 30L, 100L, 55L, - 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 5000L, 9000L, 50L, 40L, 10L, 30L, 50L, 60L, - 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L, 9000L, 5000L, 8000L, 4000L, 7000L, 3000L))) // check that the stored peak values match the expected values for ((id, metrics) <- expectedValues) { 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 5f1fe88187d1..d3886eac1523 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -92,12 +92,12 @@ class JsonProtocolSuite extends SparkFunSuite { val executorMetricsUpdate = { // Use custom accum ID for determinism val accumUpdates = - makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) - .accumulators().map(AccumulatorSuite.makeInfo) - .zipWithIndex.map { case (a, i) => a.copy(id = i) } + makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) + .accumulators().map(AccumulatorSuite.makeInfo) + .zipWithIndex.map { case (a, i) => a.copy(id = i) } val executorUpdates = new ExecutorMetrics( - Array(543L, 123456L, 123456L, 256912L, 12345L, 1234L, 123L, 12L, 432L, - 321L, 654L, 765L)) + Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), Some(executorUpdates)) } @@ -107,7 +107,7 @@ class JsonProtocolSuite extends SparkFunSuite { val stageExecutorMetrics = SparkListenerStageExecutorMetrics("1", 2, 3, new ExecutorMetrics(Array(543L, 123456L, 123456L, 256912L, 12345L, 1234L, 123L, 12L, 432L, - 321L, 654L, 765L))) + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) testEvent(taskStart, taskStartJsonString) @@ -233,7 +233,7 @@ class JsonProtocolSuite extends SparkFunSuite { hasHadoopInput = true, hasOutput = true, hasRecords = false) val newJson = JsonProtocol.taskMetricsToJson(metrics) val oldJson = newJson.removeField { case (field, _) => field == "Records Read" } - .removeField { case (field, _) => field == "Records Written" } + .removeField { case (field, _) => field == "Records Written" } val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) assert(newMetrics.inputMetrics.recordsRead == 0) assert(newMetrics.outputMetrics.recordsWritten == 0) @@ -245,7 +245,7 @@ class JsonProtocolSuite extends SparkFunSuite { hasHadoopInput = false, hasOutput = false, hasRecords = false) val newJson = JsonProtocol.taskMetricsToJson(metrics) val oldJson = newJson.removeField { case (field, _) => field == "Total Records Read" } - .removeField { case (field, _) => field == "Shuffle Records Written" } + .removeField { case (field, _) => field == "Shuffle Records Written" } val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) assert(newMetrics.shuffleReadMetrics.recordsRead == 0) assert(newMetrics.shuffleWriteMetrics.recordsWritten == 0) @@ -441,13 +441,13 @@ class JsonProtocolSuite extends SparkFunSuite { test("executorMetricsFromJson backward compatibility: handle missing metrics") { // any missing metrics should be set to 0 - val executorMetrics = new ExecutorMetrics(Array(12L, 23L, 20L, 40L, 45L, 67L, 78L, 89L, - 90L, 123L, 456L, 789L)) + val executorMetrics = new ExecutorMetrics(Array(12L, 23L, 45L, 67L, 78L, 89L, + 90L, 123L, 456L, 789L, 40L, 20L, 20L, 10L, 20L, 10L)) val oldExecutorMetricsJson = JsonProtocol.executorMetricsToJson(executorMetrics) .removeField( _._1 == "MappedPoolMemory") - val exepectedExecutorMetrics = new ExecutorMetrics(Array(12L, 23L, 20L, 40L, 45L, 67L, - 78L, 89L, 90L, 123L, 456L, 0L)) + val exepectedExecutorMetrics = new ExecutorMetrics(Array(12L, 23L, 45L, 67L, + 78L, 89L, 90L, 123L, 456L, 0L, 40L, 20L, 20L, 10L, 20L, 10L)) assertEquals(exepectedExecutorMetrics, JsonProtocol.executorMetricsFromJson(oldExecutorMetricsJson)) } @@ -459,7 +459,7 @@ class JsonProtocolSuite extends SparkFunSuite { (TestBlockId("feebo"), BlockStatus(StorageLevel.DISK_ONLY, 3L, 4L))) val blocksJson = JArray(blocks.toList.map { case (id, status) => ("Block ID" -> id.toString) ~ - ("Status" -> JsonProtocol.blockStatusToJson(status)) + ("Status" -> JsonProtocol.blockStatusToJson(status)) }) testAccumValue(Some(RESULT_SIZE), 3L, JInt(3)) testAccumValue(Some(shuffleRead.REMOTE_BLOCKS_FETCHED), 2, JInt(2)) @@ -598,7 +598,7 @@ private[spark] object JsonProtocolSuite extends Assertions { assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) }) assertOptionEquals(e1.executorUpdates, e2.executorUpdates, - (e1: ExecutorMetrics, e2: ExecutorMetrics) => assertEquals(e1, e2)) + (e1: ExecutorMetrics, e2: ExecutorMetrics) => assertEquals(e1, e2)) case (e1: SparkListenerStageExecutorMetrics, e2: SparkListenerStageExecutorMetrics) => assert(e1.execId === e2.execId) assert(e1.stageId === e2.stageId) @@ -722,12 +722,12 @@ private[spark] object JsonProtocolSuite extends Assertions { case (r1: TaskKilled, r2: TaskKilled) => assert(r1.reason == r2.reason) case (TaskCommitDenied(jobId1, partitionId1, attemptNumber1), - TaskCommitDenied(jobId2, partitionId2, attemptNumber2)) => + TaskCommitDenied(jobId2, partitionId2, attemptNumber2)) => assert(jobId1 === jobId2) assert(partitionId1 === partitionId2) assert(attemptNumber1 === attemptNumber2) case (ExecutorLostFailure(execId1, exit1CausedByApp, reason1), - ExecutorLostFailure(execId2, exit2CausedByApp, reason2)) => + ExecutorLostFailure(execId2, exit2CausedByApp, reason2)) => assert(execId1 === execId2) assert(exit1CausedByApp === exit2CausedByApp) assert(reason1 === reason2) @@ -737,8 +737,8 @@ private[spark] object JsonProtocolSuite extends Assertions { } private def assertEquals( - details1: Map[String, Seq[(String, String)]], - details2: Map[String, Seq[(String, String)]]) { + details1: Map[String, Seq[(String, String)]], + details2: Map[String, Seq[(String, String)]]) { details1.zip(details2).foreach { case ((key1, values1: Seq[(String, String)]), (key2, values2: Seq[(String, String)])) => assert(key1 === key2) @@ -780,7 +780,8 @@ private[spark] object JsonProtocolSuite extends Assertions { } } - private def assertOptionEquals[T](opt1: Option[T], + private def assertOptionEquals[T]( + opt1: Option[T], opt2: Option[T], assertEquals: (T, T) => Unit) { if (opt1.isDefined) { @@ -795,7 +796,8 @@ private[spark] object JsonProtocolSuite extends Assertions { * Use different names for methods we pass in to assertSeqEquals or assertOptionEquals */ - private def assertBlocksEquals(blocks1: Seq[(BlockId, BlockStatus)], + private def assertBlocksEquals( + blocks1: Seq[(BlockId, BlockStatus)], blocks2: Seq[(BlockId, BlockStatus)]) = { assertSeqEquals(blocks1, blocks2, assertBlockEquals) } @@ -854,7 +856,8 @@ private[spark] object JsonProtocolSuite extends Assertions { taskInfo } - private def makeAccumulableInfo(id: Int, + private def makeAccumulableInfo( + id: Int, internal: Boolean = false, countFailedValues: Boolean = false, metadata: Option[String] = None): AccumulableInfo = @@ -862,20 +865,21 @@ private[spark] object JsonProtocolSuite extends Assertions { internal, countFailedValues, metadata) /** Creates an SparkListenerExecutorMetricsUpdate event */ - private def makeExecutorMetricsUpdate(execId: String, - includeTaskMetrics: Boolean, - includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { + private def makeExecutorMetricsUpdate( + execId: String, + includeTaskMetrics: Boolean, + includeExecutorMetrics: Boolean): SparkListenerExecutorMetricsUpdate = { val taskMetrics = if (includeTaskMetrics) { Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), makeAccumulableInfo(2, false, false, None)))) - } else { + } else { Seq() } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(Array(123456L, 543L, 123456L, 256912L, 0L, 0L, 0L, 0L, 0L, - 0L, 0L, 0L))) + Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, + 0L, 0L, 0L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) } else { None } @@ -886,7 +890,8 @@ private[spark] object JsonProtocolSuite extends Assertions { * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is * set to true) or read data from a shuffle otherwise. */ - private def makeTaskMetrics(a: Long, + private def makeTaskMetrics( + a: Long, b: Long, c: Long, d: Long, @@ -943,44 +948,44 @@ private[spark] object JsonProtocolSuite extends Assertions { * ---------------------------------------- */ private val stageSubmittedJsonString = - """ - |{ - | "Event": "SparkListenerStageSubmitted", - | "Stage Info": { - | "Stage ID": 100, - | "Stage Attempt ID": 0, - | "Stage Name": "greetings", - | "Number of Tasks": 200, - | "RDD Info": [], - | "Parent IDs" : [100, 200, 300], - | "Details": "details", - | "Accumulables": [ - | { - | "ID": 2, - | "Name": "Accumulable2", - | "Update": "delta2", - | "Value": "val2", - | "Internal": false, - | "Count Failed Values": false - | }, - | { - | "ID": 1, - | "Name": "Accumulable1", - | "Update": "delta1", - | "Value": "val1", - | "Internal": false, - | "Count Failed Values": false - | } - | ] - | }, - | "Properties": { - | "France": "Paris", - | "Germany": "Berlin", - | "Russia": "Moscow", - | "Ukraine": "Kiev" - | } - |} - """.stripMargin + """ + |{ + | "Event": "SparkListenerStageSubmitted", + | "Stage Info": { + | "Stage ID": 100, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 200, + | "RDD Info": [], + | "Parent IDs" : [100, 200, 300], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2", + | "Internal": false, + | "Count Failed Values": false + | }, + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1", + | "Internal": false, + | "Count Failed Values": false + | } + | ] + | }, + | "Properties": { + | "France": "Paris", + | "Germany": "Berlin", + | "Russia": "Moscow", + | "Ukraine": "Kiev" + | } + |} + """.stripMargin private val stageCompletedJsonString = """ @@ -1833,257 +1838,257 @@ private[spark] object JsonProtocolSuite extends Assertions { private val executorAddedJsonString = s""" - |{ - | "Event": "SparkListenerExecutorAdded", - | "Timestamp": ${executorAddedTime}, - | "Executor ID": "exec1", - | "Executor Info": { - | "Host": "Hostee.awesome.com", - | "Total Cores": 11, - | "Log Urls" : { - | "stderr" : "mystderr", - | "stdout" : "mystdout" - | } - | } - |} + |{ + | "Event": "SparkListenerExecutorAdded", + | "Timestamp": ${executorAddedTime}, + | "Executor ID": "exec1", + | "Executor Info": { + | "Host": "Hostee.awesome.com", + | "Total Cores": 11, + | "Log Urls" : { + | "stderr" : "mystderr", + | "stdout" : "mystdout" + | } + | } + |} """.stripMargin private val executorRemovedJsonString = s""" - |{ - | "Event": "SparkListenerExecutorRemoved", - | "Timestamp": ${executorRemovedTime}, - | "Executor ID": "exec2", - | "Removed Reason": "test reason" - |} + |{ + | "Event": "SparkListenerExecutorRemoved", + | "Timestamp": ${executorRemovedTime}, + | "Executor ID": "exec2", + | "Removed Reason": "test reason" + |} """.stripMargin private val executorMetricsUpdateJsonString = s""" - |{ - | "Event": "SparkListenerExecutorMetricsUpdate", - | "Executor ID": "exec3", - | "Metrics Updated": [ - | { - | "Task ID": 1, - | "Stage ID": 2, - | "Stage Attempt ID": 3, - | "Accumulator Updates": [ - | { - | "ID": 0, - | "Name": "$EXECUTOR_DESERIALIZE_TIME", - | "Update": 300, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 1, - | "Name": "$EXECUTOR_DESERIALIZE_CPU_TIME", - | "Update": 300, - | "Internal": true, - | "Count Failed Values": true - | }, - | + |{ + | "Event": "SparkListenerExecutorMetricsUpdate", + | "Executor ID": "exec3", + | "Metrics Updated": [ + | { + | "Task ID": 1, + | "Stage ID": 2, + | "Stage Attempt ID": 3, + | "Accumulator Updates": [ + | { + | "ID": 0, + | "Name": "$EXECUTOR_DESERIALIZE_TIME", + | "Update": 300, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 1, + | "Name": "$EXECUTOR_DESERIALIZE_CPU_TIME", + | "Update": 300, + | "Internal": true, + | "Count Failed Values": true + | }, + | + | { + | "ID": 2, + | "Name": "$EXECUTOR_RUN_TIME", + | "Update": 400, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 3, + | "Name": "$EXECUTOR_CPU_TIME", + | "Update": 400, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 4, + | "Name": "$RESULT_SIZE", + | "Update": 500, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 5, + | "Name": "$JVM_GC_TIME", + | "Update": 600, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 6, + | "Name": "$RESULT_SERIALIZATION_TIME", + | "Update": 700, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 7, + | "Name": "$MEMORY_BYTES_SPILLED", + | "Update": 800, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 8, + | "Name": "$DISK_BYTES_SPILLED", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 9, + | "Name": "$PEAK_EXECUTION_MEMORY", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 10, + | "Name": "$UPDATED_BLOCK_STATUSES", + | "Update": [ + | { + | "Block ID": "rdd_0_0", + | "Status": { + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Deserialized": false, + | "Replication": 2 + | }, + | "Memory Size": 0, + | "Disk Size": 0 + | } + | } + | ], + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 11, + | "Name": "${shuffleRead.REMOTE_BLOCKS_FETCHED}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 12, + | "Name": "${shuffleRead.LOCAL_BLOCKS_FETCHED}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 13, + | "Name": "${shuffleRead.REMOTE_BYTES_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 14, + | "Name": "${shuffleRead.REMOTE_BYTES_READ_TO_DISK}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 15, + | "Name": "${shuffleRead.LOCAL_BYTES_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 16, + | "Name": "${shuffleRead.FETCH_WAIT_TIME}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 17, + | "Name": "${shuffleRead.RECORDS_READ}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 18, + | "Name": "${shuffleWrite.BYTES_WRITTEN}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 19, + | "Name": "${shuffleWrite.RECORDS_WRITTEN}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 20, + | "Name": "${shuffleWrite.WRITE_TIME}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 21, + | "Name": "${input.BYTES_READ}", + | "Update": 2100, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 22, + | "Name": "${input.RECORDS_READ}", + | "Update": 21, + | "Internal": true, + | "Count Failed Values": true + | }, | { - | "ID": 2, - | "Name": "$EXECUTOR_RUN_TIME", - | "Update": 400, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 3, - | "Name": "$EXECUTOR_CPU_TIME", - | "Update": 400, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 4, - | "Name": "$RESULT_SIZE", - | "Update": 500, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 5, - | "Name": "$JVM_GC_TIME", - | "Update": 600, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 6, - | "Name": "$RESULT_SERIALIZATION_TIME", - | "Update": 700, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 7, - | "Name": "$MEMORY_BYTES_SPILLED", - | "Update": 800, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 8, - | "Name": "$DISK_BYTES_SPILLED", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 9, - | "Name": "$PEAK_EXECUTION_MEMORY", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 10, - | "Name": "$UPDATED_BLOCK_STATUSES", - | "Update": [ - | { - | "Block ID": "rdd_0_0", - | "Status": { - | "Storage Level": { - | "Use Disk": true, - | "Use Memory": true, - | "Deserialized": false, - | "Replication": 2 - | }, - | "Memory Size": 0, - | "Disk Size": 0 - | } - | } - | ], - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 11, - | "Name": "${shuffleRead.REMOTE_BLOCKS_FETCHED}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 12, - | "Name": "${shuffleRead.LOCAL_BLOCKS_FETCHED}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 13, - | "Name": "${shuffleRead.REMOTE_BYTES_READ}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 14, - | "Name": "${shuffleRead.REMOTE_BYTES_READ_TO_DISK}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 15, - | "Name": "${shuffleRead.LOCAL_BYTES_READ}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 16, - | "Name": "${shuffleRead.FETCH_WAIT_TIME}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 17, - | "Name": "${shuffleRead.RECORDS_READ}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 18, - | "Name": "${shuffleWrite.BYTES_WRITTEN}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 19, - | "Name": "${shuffleWrite.RECORDS_WRITTEN}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 20, - | "Name": "${shuffleWrite.WRITE_TIME}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 21, - | "Name": "${input.BYTES_READ}", - | "Update": 2100, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 22, - | "Name": "${input.RECORDS_READ}", - | "Update": 21, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 23, - | "Name": "${output.BYTES_WRITTEN}", - | "Update": 1200, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 24, - | "Name": "${output.RECORDS_WRITTEN}", - | "Update": 12, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 25, - | "Name": "$TEST_ACCUM", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | } - | ] - | } - | ], - | "Executor Metrics Updated" : { - | "JVMHeapMemory" : 543, - | "JVMOffHeapMemory" : 123456, - | "ProcessTreeRSSMemory": 123456, - | "ProcessTreeVMemory": 256912, - | "OnHeapExecutionMemory" : 12345, - | "OffHeapExecutionMemory" : 1234, - | "OnHeapStorageMemory" : 123, - | "OffHeapStorageMemory" : 12, - | "OnHeapUnifiedMemory" : 432, - | "OffHeapUnifiedMemory" : 321, - | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765 - | } - | + | "ID": 23, + | "Name": "${output.BYTES_WRITTEN}", + | "Update": 1200, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 24, + | "Name": "${output.RECORDS_WRITTEN}", + | "Update": 12, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 25, + | "Name": "$TEST_ACCUM", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | } + | ] + | } + | ], + | "Executor Metrics Updated" : { + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "ProcessTreeRSSMemory": 123456, + | "ProcessTreeVMemory": 256912, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765 + | } + | |} """.stripMargin @@ -2097,8 +2102,6 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Executor Metrics" : { | "JVMHeapMemory" : 543, | "JVMOffHeapMemory" : 123456, - | "ProcessTreeRSSMemory": 123456, - | "ProcessTreeVMemory": 256912, | "OnHeapExecutionMemory" : 12345, | "OffHeapExecutionMemory" : 1234, | "OnHeapStorageMemory" : 123, @@ -2106,7 +2109,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "OnHeapUnifiedMemory" : 432, | "OffHeapUnifiedMemory" : 321, | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765 + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 | } |} """.stripMargin @@ -2136,36 +2145,36 @@ private[spark] object JsonProtocolSuite extends Assertions { private val executorBlacklistedJsonString = s""" - |{ - | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorBlacklisted", - | "time" : ${executorBlacklistedTime}, - | "executorId" : "exec1", - | "taskFailures" : 22 - |} + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorBlacklisted", + | "time" : ${executorBlacklistedTime}, + | "executorId" : "exec1", + | "taskFailures" : 22 + |} """.stripMargin private val executorUnblacklistedJsonString = s""" - |{ - | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted", - | "time" : ${executorUnblacklistedTime}, - | "executorId" : "exec1" - |} + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted", + | "time" : ${executorUnblacklistedTime}, + | "executorId" : "exec1" + |} """.stripMargin private val nodeBlacklistedJsonString = s""" - |{ - | "Event" : "org.apache.spark.scheduler.SparkListenerNodeBlacklisted", - | "time" : ${nodeBlacklistedTime}, - | "hostId" : "node1", - | "executorFailures" : 33 - |} + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeBlacklisted", + | "time" : ${nodeBlacklistedTime}, + | "hostId" : "node1", + | "executorFailures" : 33 + |} """.stripMargin private val nodeUnblacklistedJsonString = s""" - |{ - | "Event" : "org.apache.spark.scheduler.SparkListenerNodeUnblacklisted", - | "time" : ${nodeUnblacklistedTime}, - | "hostId" : "node1" - |} + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeUnblacklisted", + | "time" : ${nodeUnblacklistedTime}, + | "hostId" : "node1" + |} """.stripMargin } From 03cd5bceddc7867a90918430b23bf9fa3771edfd Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Mon, 13 Aug 2018 16:30:20 -0700 Subject: [PATCH 26/29] code review comments --- .../apache/spark/executor/ExecutorMetrics.scala | 4 ++-- .../org/apache/spark/memory/MemoryManager.scala | 16 ++++++++++++---- .../spark/metrics/ExecutorMetricType.scala | 2 +- .../apache/spark/status/AppStatusListener.scala | 4 ++-- 4 files changed, 17 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 36544cf65ec6..2933f3ba6d3b 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -68,10 +68,10 @@ class ExecutorMetrics private[spark] extends Serializable { * @return if there is a new peak value for any metric */ private[spark] def compareAndUpdatePeakValues(executorMetrics: ExecutorMetrics): Boolean = { - var updated: Boolean = false + var updated = false (0 until ExecutorMetricType.values.length).foreach { idx => - if ( executorMetrics.metrics(idx) > metrics(idx)) { + if (executorMetrics.metrics(idx) > metrics(idx)) { updated = true metrics(idx) = executorMetrics.metrics(idx) } diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 965515b8abe1..4fde2d0beaa7 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -183,22 +183,30 @@ private[spark] abstract class MemoryManager( /** * On heap execution memory currently in use, in bytes. */ - final def onHeapExecutionMemoryUsed: Long = onHeapExecutionMemoryPool.memoryUsed + final def onHeapExecutionMemoryUsed: Long = synchronized { + onHeapExecutionMemoryPool.memoryUsed + } /** * Off heap execution memory currently in use, in bytes. */ - final def offHeapExecutionMemoryUsed: Long = offHeapExecutionMemoryPool.memoryUsed + final def offHeapExecutionMemoryUsed: Long = synchronized { + offHeapExecutionMemoryPool.memoryUsed + } /** * On heap storage memory currently in use, in bytes. */ - final def onHeapStorageMemoryUsed: Long = onHeapStorageMemoryPool.memoryUsed + final def onHeapStorageMemoryUsed: Long = synchronized { + onHeapStorageMemoryPool.memoryUsed + } /** * Off heap storage memory currently in use, in bytes. */ - final def offHeapStorageMemoryUsed: Long = offHeapStorageMemoryPool.memoryUsed + final def offHeapStorageMemoryUsed: Long = synchronized { + offHeapStorageMemoryPool.memoryUsed + } /** * Returns the execution memory consumption, in bytes, for the given task. diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index 456663e017c2..cd10dad25e87 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -36,7 +36,7 @@ private[spark] abstract class MemoryManagerExecutorMetricType( } } -private[spark]abstract class MBeanExecutorMetricType(mBeanName: String) +private[spark] abstract class MBeanExecutorMetricType(mBeanName: String) extends ExecutorMetricType { private val bean = ManagementFactory.newPlatformMXBeanProxy( ManagementFactory.getPlatformMBeanServer, 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 a0f0c4ae4d59..6a4ce3de28cb 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -690,8 +690,8 @@ private[spark] class AppStatusListener( // check if there is a new peak value for any of the executor level memory metrics // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed // for the live UI. - event.executorUpdates.foreach { updates: ExecutorMetrics => - liveExecutors.get(event.execId).foreach { exec: LiveExecutor => + event.executorUpdates.foreach { updates => + liveExecutors.get(event.execId).foreach { exec => if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(updates)) { maybeUpdate(exec, now) } From a14b82a39fa00c43fa60c245f62a4fb0c154bd9a Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Tue, 14 Aug 2018 10:54:08 -0700 Subject: [PATCH 27/29] merge conflicts --- core/src/main/scala/org/apache/spark/SparkContext.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index cecbb910fa6e..d4fabd7f7525 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1962,13 +1962,6 @@ class SparkContext(config: SparkConf) extends Logging { } _heartbeater = null } - if (_dagScheduler != null) { - Utils.tryLogNonFatalError { - _dagScheduler.stop() - } - _dagScheduler = null - } - if (env != null && _heartbeatReceiver != null) { Utils.tryLogNonFatalError { env.rpcEnv.stop(_heartbeatReceiver) From 2897281a384d25556609a17be21f926cb5d68dd6 Mon Sep 17 00:00:00 2001 From: Edwina Lu Date: Thu, 16 Aug 2018 13:13:14 -0700 Subject: [PATCH 28/29] disable stage executor metrics logging by default --- .../main/scala/org/apache/spark/executor/Executor.scala | 7 ++++++- .../scala/org/apache/spark/internal/config/package.scala | 2 +- .../apache/spark/deploy/history/HistoryServerSuite.scala | 1 + .../apache/spark/scheduler/EventLoggingListenerSuite.scala | 1 + 4 files changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 6c67e640b338..a3e712a29061 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -217,7 +217,12 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() - heartbeater.stop() + try { + heartbeater.stop() + } catch { + case NonFatal(e) => + logWarning("Unable to stop heartbeater", e) + } threadPool.shutdown() if (!isLocal) { env.stop() diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 87b5d3633baf..6d521d6c9548 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -72,7 +72,7 @@ package object config { private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS = ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled") .booleanConf - .createWithDefault(true) + .createWithDefault(false) private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) 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 3b06126ee9a3..11a2db81f7c6 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 @@ -82,6 +82,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers .set("spark.history.fs.update.interval", "0") .set("spark.testing", "true") .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) + .set("spark.eventLog.logStageExecutorMetrics.enabled", "true") conf.setAll(extraConf) provider = new FsHistoryProvider(conf) provider.checkForLogs() 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 10ced5b27b6a..cecd6996df7b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -517,6 +517,7 @@ object EventLoggingListenerSuite { conf.set("spark.eventLog.compress", "true") conf.set("spark.io.compression.codec", codec) } + conf.set("spark.eventLog.logStageExecutorMetrics.enabled", "true") conf } From b14cebc7d3872613f66a415b0b1943b6838ca9cf Mon Sep 17 00:00:00 2001 From: Reza Safi Date: Fri, 17 Aug 2018 13:13:52 -0400 Subject: [PATCH 29/29] Update JsonProtocolSuite with new metrics. --- .../apache/spark/util/JsonProtocolSuite.scala | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) 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 d3886eac1523..b2ebfc464252 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -92,9 +92,9 @@ class JsonProtocolSuite extends SparkFunSuite { val executorMetricsUpdate = { // Use custom accum ID for determinism val accumUpdates = - makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) - .accumulators().map(AccumulatorSuite.makeInfo) - .zipWithIndex.map { case (a, i) => a.copy(id = i) } + makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true) + .accumulators().map(AccumulatorSuite.makeInfo) + .zipWithIndex.map { case (a, i) => a.copy(id = i) } val executorUpdates = new ExecutorMetrics( Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)) @@ -106,7 +106,7 @@ class JsonProtocolSuite extends SparkFunSuite { "In your multitude...", 300), RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100L, 0L)) val stageExecutorMetrics = SparkListenerStageExecutorMetrics("1", 2, 3, - new ExecutorMetrics(Array(543L, 123456L, 123456L, 256912L, 12345L, 1234L, 123L, 12L, 432L, + new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -2077,8 +2077,6 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Executor Metrics Updated" : { | "JVMHeapMemory" : 543, | "JVMOffHeapMemory" : 123456, - | "ProcessTreeRSSMemory": 123456, - | "ProcessTreeVMemory": 256912, | "OnHeapExecutionMemory" : 12345, | "OffHeapExecutionMemory" : 1234, | "OnHeapStorageMemory" : 123, @@ -2086,7 +2084,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "OnHeapUnifiedMemory" : 432, | "OffHeapUnifiedMemory" : 321, | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765 + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 | } | |}