From 9f5b195ea486cef667d620ccce5b6917d830d646 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Fri, 4 Jan 2019 11:49:05 -0800 Subject: [PATCH 01/22] [SPARK-26329][CORE] Faster polling of executor memory metrics. Prior to this change, in an executor, on each heartbeat, memory metrics are polled and sent in the heartbeat. The heartbeat interval is 10s by default. With this change, in an executor, memory metrics can optionally be polled in a separate poller at an independent interval. We use a map of (stageId, stageAttemptId) to number of running tasks in the executor to track what stages are active. We track the peak per stage for each metric by keeping a map of (stageId, stageAttemptId) to memory metric peaks, and update the peaks each time we poll. We send the peaks in the heartbeat; this is now a map of (stageId, stageAttemptId) to executor metrics. We also keep a map of taskId to memory metric peaks in the executor. This tracks the metric peaks during the lifetime of the task. At end of a task, we send the peak metric values in the task result. For the driver, we still only poll on heartbeats. --- .../org/apache/spark/HeartbeatReceiver.scala | 7 +- .../scala/org/apache/spark/Heartbeater.scala | 22 --- .../scala/org/apache/spark/SparkContext.scala | 11 +- .../org/apache/spark/executor/Executor.scala | 141 +++++++++++++++++- .../spark/executor/ExecutorMetrics.scala | 33 +++- .../spark/internal/config/package.scala | 5 + .../spark/metrics/ExecutorMetricType.scala | 1 - .../apache/spark/scheduler/DAGScheduler.scala | 12 +- .../spark/scheduler/DAGSchedulerEvent.scala | 1 + .../scheduler/EventLoggingListener.scala | 31 +++- .../spark/scheduler/SparkListener.scala | 7 +- .../apache/spark/scheduler/TaskResult.scala | 24 ++- .../spark/scheduler/TaskScheduler.scala | 4 +- .../spark/scheduler/TaskSchedulerImpl.scala | 5 +- .../spark/scheduler/TaskSetManager.scala | 8 +- .../spark/status/AppStatusListener.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 25 +++- .../ExecutorAllocationManagerSuite.scala | 22 +-- .../apache/spark/HeartbeatReceiverSuite.scala | 3 +- .../spark/scheduler/DAGSchedulerSuite.scala | 23 +-- .../scheduler/EventLoggingListenerSuite.scala | 32 ++-- .../ExternalClusterManagerSuite.scala | 4 +- .../spark/scheduler/ReplayListenerSuite.scala | 4 +- .../scheduler/SchedulerIntegrationSuite.scala | 2 +- .../scheduler/TaskSchedulerImplSuite.scala | 4 +- .../spark/scheduler/TaskSetManagerSuite.scala | 22 ++- .../spark/status/AppStatusListenerSuite.scala | 70 +++++---- .../org/apache/spark/ui/StagePageSuite.scala | 6 +- .../apache/spark/util/JsonProtocolSuite.scala | 140 +++++++++++++---- 29 files changed, 488 insertions(+), 185 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index f7e3103f2502..1f4ea56a8c2e 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -19,7 +19,8 @@ package org.apache.spark import java.util.concurrent.{ScheduledFuture, TimeUnit} -import scala.collection.mutable +import scala.collection.Map +import scala.collection.mutable.HashMap import scala.concurrent.Future import org.apache.spark.executor.ExecutorMetrics @@ -40,7 +41,7 @@ private[spark] case class Heartbeat( executorId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates blockManagerId: BlockManagerId, - executorUpdates: ExecutorMetrics) // executor level updates + executorUpdates: Map[(Int, Int), ExecutorMetrics]) // executor level updates /** * An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is @@ -73,7 +74,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) private[spark] var scheduler: TaskScheduler = null // executor ID -> timestamp of when the last heartbeat from this executor was received - private val executorLastSeen = new mutable.HashMap[String, Long] + private val executorLastSeen = new HashMap[String, Long] private val executorTimeoutMs = sc.conf.get(config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT) diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 1012755e068d..4f749dd05bdd 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -19,23 +19,18 @@ 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.ExecutorMetricType 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( - memoryManager: MemoryManager, reportHeartbeat: () => Unit, name: String, intervalMs: Long) extends Logging { @@ -58,21 +53,4 @@ private[spark] class Heartbeater( heartbeater.shutdown() heartbeater.awaitTermination(10, TimeUnit.SECONDS) } - - /** - * Get the current executor level metrics. These are returned as an array, with the index - * determined by ExecutorMetricType.metricToOffset - */ - def getCurrentMetrics(): ExecutorMetrics = { - - val metrics = new Array[Long](ExecutorMetricType.numMetrics) - var offset = 0 - ExecutorMetricType.metricGetters.foreach { metric => - val newMetrics = metric.getMetricValues(memoryManager) - Array.copy(newMetrics, 0, metrics, offset, newMetrics.size) - offset += newMetrics.length - } - new ExecutorMetrics(metrics) - } } - diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 75182b0c9008..96b69d40e1d0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -42,6 +42,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -516,7 +517,7 @@ class SparkContext(config: SparkConf) extends Logging { _heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet) // create and start the heartbeater for collecting memory metrics - _heartbeater = new Heartbeater(env.memoryManager, + _heartbeater = new Heartbeater( () => SparkContext.this.reportHeartBeat(), "driver-heartbeater", conf.get(EXECUTOR_HEARTBEAT_INTERVAL)) @@ -2425,10 +2426,14 @@ class SparkContext(config: SparkConf) extends Logging { /** Reports heartbeat metrics for the driver. */ private def reportHeartBeat(): Unit = { - val driverUpdates = _heartbeater.getCurrentMetrics() + val currentMetrics = ExecutorMetrics.getCurrentMetrics(env.memoryManager) + val driverUpdates = new HashMap[(Int, Int), ExecutorMetrics] + // In the driver, we do not track per-stage metrics, so use a dummy stage + // for the key + driverUpdates.put((-1, -1), new ExecutorMetrics(currentMetrics)) val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, - Some(driverUpdates))) + 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/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index d5c5f47a8c71..098ee291c431 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -18,12 +18,14 @@ package org.apache.spark.executor import java.io.{File, NotSerializableException} +import java.lang.Long.{MAX_VALUE => LONG_MAX_VALUE} import java.lang.Thread.UncaughtExceptionHandler import java.lang.management.ManagementFactory import java.net.{URI, URL} import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent._ +import java.util.concurrent.atomic.{AtomicLong, AtomicLongArray} import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ @@ -38,6 +40,7 @@ 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.metrics.ExecutorMetricType import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ @@ -170,6 +173,17 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] + type StageKey = (Int, Int) + + // Map of (stageId, stageAttemptId) to count of running tasks + private val activeStages = new ConcurrentHashMap[StageKey, AtomicLong] + + // Map of (stageId, stageAttemptId) to executor metric peaks + private val stageMetricPeaks = new ConcurrentHashMap[StageKey, AtomicLongArray] + + // Map of taskId to executor metric peaks + private val taskMetricPeaks = new ConcurrentHashMap[Long, AtomicLongArray] + /** * When an executor is unable to send heartbeats to the driver more than `HEARTBEAT_MAX_FAILURES` * times, it should kill itself. The default value is 60. For example, if max failures is 60 and @@ -188,9 +202,44 @@ private[spark] class Executor( */ private val HEARTBEAT_INTERVAL_MS = conf.get(EXECUTOR_HEARTBEAT_INTERVAL) + /** + * Interval to poll for executor metrics, in milliseconds + */ + private val METRICS_POLLING_INTERVAL_MS = conf.get(EXECUTOR_METRICS_POLLING_INTERVAL) + + // Executor for the metrics polling task + private val poller = + if (METRICS_POLLING_INTERVAL_MS > 0) { + ThreadUtils.newDaemonSingleThreadScheduledExecutor("executor-poller") + } else { + null + } + + private def poll(): Unit = { + // get the latest values for the metrics + val latestMetrics = ExecutorMetrics.getCurrentMetrics(env.memoryManager) + + def compareAndUpdate(current: Long, latest: Long): Long = + if (latest > current) latest else current + + def updatePeaks(metrics: AtomicLongArray): Unit = { + (0 until metrics.length).foreach { i => + metrics.getAndAccumulate(i, latestMetrics(i), compareAndUpdate) + } + } + + def peaksForStage(k: StageKey, v: AtomicLong): AtomicLongArray = + if (v.get() > 0) stageMetricPeaks.get(k) else null + + // for each active stage (number of running tasks > 0), get the peaks and update them + activeStages.forEach[AtomicLongArray](LONG_MAX_VALUE, peaksForStage, updatePeaks) + + // for each running task, update the peaks + taskMetricPeaks.forEachValue(LONG_MAX_VALUE, updatePeaks) + } + // Executor for the heartbeat task. private val heartbeater = new Heartbeater( - env.memoryManager, () => Executor.this.reportHeartBeat(), "executor-heartbeater", HEARTBEAT_INTERVAL_MS) @@ -207,11 +256,19 @@ private[spark] class Executor( heartbeater.start() + if (poller != null) { + val pollingTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(poll()) + } + poller.scheduleAtFixedRate(pollingTask, 0L, METRICS_POLLING_INTERVAL_MS, TimeUnit.MILLISECONDS) + } + private[executor] def numRunningTasks: Int = runningTasks.size() def launchTask(context: ExecutorBackend, taskDescription: TaskDescription): Unit = { val tr = new TaskRunner(context, taskDescription) runningTasks.put(taskDescription.taskId, tr) + taskMetricPeaks.put(taskDescription.taskId, new AtomicLongArray(ExecutorMetricType.numMetrics)) threadPool.execute(tr) } @@ -254,12 +311,21 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() + if (poller != null) { + try { + poller.shutdown() + poller.awaitTermination(METRICS_POLLING_INTERVAL_MS, TimeUnit.MILLISECONDS) + } catch { + case NonFatal(e) => + logWarning("Unable to stop poller", e) + } + } try { heartbeater.stop() } catch { case NonFatal(e) => logWarning("Unable to stop heartbeater", e) - } + } threadPool.shutdown() // Notify plugins that executor is shutting down so they can terminate cleanly @@ -412,6 +478,21 @@ private[spark] class Executor( env.mapOutputTracker.asInstanceOf[MapOutputTrackerWorker].updateEpoch(task.epoch) } + val stageId = task.stageId + val stageAttemptId = task.stageAttemptId + val count = new AtomicLong(0) + val old = activeStages.putIfAbsent((stageId, stageAttemptId), count) + val stageCount = + if (old != null) { + old.incrementAndGet() + } else { + logDebug(s"added ($stageId, $stageAttemptId) to activeStages") + count.incrementAndGet() + } + logDebug(s"activeStages: ($stageId, $stageAttemptId) -> $stageCount") + stageMetricPeaks.putIfAbsent((stageId, stageAttemptId), + new AtomicLongArray(ExecutorMetricType.numMetrics)) + // Run the actual task and measure its runtime. taskStartTimeNs = System.nanoTime() taskStartCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { @@ -527,10 +608,19 @@ private[spark] class Executor( executorSource.METRIC_DISK_BYTES_SPILLED.inc(task.metrics.diskBytesSpilled) executorSource.METRIC_MEMORY_BYTES_SPILLED.inc(task.metrics.memoryBytesSpilled) + def getMetricPeaks(): Array[Long] = { + val currentPeaks = taskMetricPeaks.get(taskId) + val metricPeaks = new Array[Long](ExecutorMetricType.numMetrics) + ExecutorMetricType.metricToOffset.foreach { case (_, i) => + metricPeaks(i) = currentPeaks.get(i) + } + metricPeaks + } + // Note: accumulator updates must be collected after TaskMetrics is updated val accumUpdates = task.collectAccumulatorUpdates() // TODO: do not serialize value twice - val directResult = new DirectTaskResult(valueBytes, accumUpdates) + val directResult = new DirectTaskResult(valueBytes, accumUpdates, getMetricPeaks) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit() @@ -556,10 +646,31 @@ private[spark] class Executor( } } + def decrementCount(stage: StageKey, count: AtomicLong): AtomicLong = { + val countValue = count.decrementAndGet() + if (countValue == 0L) { + logDebug(s"removing (${stage._1}, ${stage._2}) from activeStages") + null + } else { + logDebug(s"activeStages: (${stage._1}, ${stage._2}) -> " + countValue) + count + } + } + + // If the count is zero, the stage is removed from activeStages + activeStages.compute((stageId, stageAttemptId), decrementCount) + + // If the stage has been removed from activeStages, remove it from stageMetricPeaks too + def removeInactive(k: StageKey): Unit = { + if (activeStages.get(k) == null) { + stageMetricPeaks.remove(k) + } + } + stageMetricPeaks.forEachKey(LONG_MAX_VALUE, removeInactive) + executorSource.SUCCEEDED_TASKS.inc(1L) setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) - } catch { case t: TaskKilledException => logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}") @@ -632,6 +743,7 @@ private[spark] class Executor( } } finally { runningTasks.remove(taskId) + taskMetricPeaks.remove(taskId) } } @@ -848,8 +960,25 @@ private[spark] class Executor( val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() val curGCTime = computeTotalGcTime() - // get executor level memory metrics - val executorUpdates = heartbeater.getCurrentMetrics() + // if not polling in a separater poller, poll here + if (poller == null) { + poll() + } + + // build the executor level memory metrics + val executorUpdates = new HashMap[StageKey, ExecutorMetrics] + + def peaksForStage(k: StageKey, v: AtomicLong): (StageKey, AtomicLongArray) = + if (v.get() > 0) (k, stageMetricPeaks.get(k)) else null + + def addPeaks(nested: (StageKey, AtomicLongArray)): Unit = { + val (k, v) = nested + executorUpdates.put(k, new ExecutorMetrics(v)) + // at the same time, reset the peaks in stageMetricPeaks + stageMetricPeaks.put(k, new AtomicLongArray(ExecutorMetricType.numMetrics)) + } + + activeStages.forEach[(StageKey, AtomicLongArray)](LONG_MAX_VALUE, peaksForStage, addPeaks) for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { 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 f19ac813fde3..78adfe632635 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -16,7 +16,10 @@ */ package org.apache.spark.executor +import java.util.concurrent.atomic.AtomicLongArray + import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.memory.MemoryManager import org.apache.spark.metrics.ExecutorMetricType /** @@ -46,6 +49,13 @@ class ExecutorMetrics private[spark] extends Serializable { Array.copy(metrics, 0, this.metrics, 0, Math.min(metrics.size, this.metrics.size)) } + private[spark] def this(metrics: AtomicLongArray) { + this() + ExecutorMetricType.metricToOffset.foreach { case (_, i) => + this.metrics(i) = metrics.get(i) + } + } + /** * Constructor: create the ExecutorMetrics with using a given map. * @@ -53,7 +63,7 @@ class ExecutorMetrics private[spark] extends Serializable { */ private[spark] def this(executorMetrics: Map[String, Long]) { this() - ExecutorMetricType.metricToOffset.foreach { case(name, idx) => + ExecutorMetricType.metricToOffset.foreach { case (name, idx) => metrics(idx) = executorMetrics.getOrElse(name, 0L) } } @@ -76,3 +86,24 @@ class ExecutorMetrics private[spark] extends Serializable { updated } } + +private[spark] object ExecutorMetrics { + + /** + * Get the current executor metrics. These are returned as an array, with the index + * determined by ExecutorMetricType.metricToOffset. + * + * @param memoryManager the memory manager for execution and storage memory + * @return the values of the metrics + */ + def getCurrentMetrics(memoryManager: MemoryManager): Array[Long] = { + val currentMetrics = new Array[Long](ExecutorMetricType.numMetrics) + var offset = 0 + ExecutorMetricType.metricGetters.foreach { metricType => + val metricValues = metricType.getMetricValues(memoryManager) + Array.copy(metricValues, 0, currentMetrics, offset, metricValues.length) + offset += metricValues.length + } + currentMetrics + } +} 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 f2b88fe00cdf..26bc969a2ac8 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 @@ -176,6 +176,11 @@ package object config { private[spark] val EXECUTOR_HEARTBEAT_MAX_FAILURES = ConfigBuilder("spark.executor.heartbeat.maxFailures").internal().intConf.createWithDefault(60) + private[spark] val EXECUTOR_METRICS_POLLING_INTERVAL = + ConfigBuilder("spark.executor.metrics.pollingInterval") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("0") + private[spark] val EXECUTOR_JAVA_OPTIONS = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS) .withPrepended(SparkLauncher.EXECUTOR_DEFAULT_JAVA_OPTIONS) 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 b830f9f402eb..76e281393e35 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -201,7 +201,6 @@ private[spark] object ExecutorMetricType { GarbageCollectionMetrics ) - val (metricToOffset, numMetrics) = { var numberOfMetrics = 0 val definedMetricsAndOffset = mutable.LinkedHashMap.empty[String, Int] 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 5072e617937e..afb4c88be5e5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -248,9 +248,10 @@ private[spark] class DAGScheduler( reason: TaskEndReason, result: Any, accumUpdates: Seq[AccumulatorV2[_, _]], + metricPeaks: Array[Long], taskInfo: TaskInfo): Unit = { eventProcessLoop.post( - CompletionEvent(task, reason, result, accumUpdates, taskInfo)) + CompletionEvent(task, reason, result, accumUpdates, metricPeaks, taskInfo)) } /** @@ -263,10 +264,10 @@ private[spark] class DAGScheduler( // (taskId, stageId, stageAttemptId, accumUpdates) accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId, - // executor metrics indexed by ExecutorMetricType.values - executorUpdates: ExecutorMetrics): Boolean = { + // (stageId, stageAttemptId) -> metrics + executorUpdates: Map[(Int, Int), ExecutorMetrics]): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, - Some(executorUpdates))) + executorUpdates)) blockManagerMaster.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(10.minutes, "BlockManagerHeartbeat")) } @@ -1310,7 +1311,8 @@ private[spark] class DAGScheduler( } listenerBus.post(SparkListenerTaskEnd(event.task.stageId, event.task.stageAttemptId, - Utils.getFormattedClassName(event.task), event.reason, event.taskInfo, taskMetrics)) + Utils.getFormattedClassName(event.task), event.reason, event.taskInfo, + new ExecutorMetrics(event.metricPeaks), taskMetrics)) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index b514c2e7056f..78d458338e8f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -76,6 +76,7 @@ private[scheduler] case class CompletionEvent( reason: TaskEndReason, result: Any, accumUpdates: Seq[AccumulatorV2[_, _]], + metricPeaks: Array[Long], taskInfo: TaskInfo) extends DAGSchedulerEvent 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 20c74b16ce08..5f196bbe31b9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -173,7 +173,19 @@ private[spark] class EventLoggingListener( override def onTaskGettingResult(event: SparkListenerTaskGettingResult): Unit = logEvent(event) - override def onTaskEnd(event: SparkListenerTaskEnd): Unit = logEvent(event) + override def onTaskEnd(event: SparkListenerTaskEnd): Unit = { + logEvent(event) + if (shouldLogStageExecutorMetrics) { + val k1 = (event.stageId, event.stageAttemptId) + liveStageExecutorMetrics.foreach { case (k2, peakExecutorMetrics) => + if (k1 == k2) { + val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + event.taskInfo.executorId, new ExecutorMetrics()) + peakMetrics.compareAndUpdatePeakValues(event.taskExecutorMetrics) + } + } + } + } override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = { logEvent(redactEvent(event)) @@ -268,12 +280,17 @@ 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 - event.executorUpdates.foreach { executorUpdates => - liveStageExecutorMetrics.values.foreach { peakExecutorMetrics => - val peakMetrics = peakExecutorMetrics.getOrElseUpdate( - event.execId, new ExecutorMetrics()) - peakMetrics.compareAndUpdatePeakValues(executorUpdates) + event.executorUpdates.foreach { case (k1, peakUpdates) => + liveStageExecutorMetrics.foreach { case (k2, peakExecutorMetrics) => + // If the update came from the driver, the key k1 will be the dummy key (-1, -1), + // so record those peaks for all active stages. + // Otherwise, record the peaks for the matching stage. + val k0 = (-1, -1) + if (k1 == k0 || k1 == k2) { + val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + event.execId, new ExecutorMetrics()) + peakMetrics.compareAndUpdatePeakValues(peakUpdates) + } } } } 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 666ce3dc7c16..c150b0341500 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -64,6 +64,7 @@ case class SparkListenerTaskEnd( taskType: String, reason: TaskEndReason, taskInfo: TaskInfo, + taskExecutorMetrics: ExecutorMetrics, // may be null if the task has failed @Nullable taskMetrics: TaskMetrics) extends SparkListenerEvent @@ -162,13 +163,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 + * @param executorUpdates executor level per-stage metrics updates */ @DeveloperApi case class SparkListenerExecutorMetricsUpdate( execId: String, accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])], - executorUpdates: Option[ExecutorMetrics] = None) + executorUpdates: Map[(Int, Int), ExecutorMetrics] = Map.empty) extends SparkListenerEvent /** @@ -177,7 +178,7 @@ case class SparkListenerExecutorMetricsUpdate( * @param execId executor id * @param stageId stage id * @param stageAttemptId stage attempt - * @param executorMetrics executor level metrics, indexed by ExecutorMetricType.values + * @param executorMetrics executor level metrics peak values */ @DeveloperApi case class SparkListenerStageExecutorMetrics( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 836769e1723d..289784979ff0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -23,33 +23,38 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkEnv +import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.serializer.SerializerInstance import org.apache.spark.storage.BlockId import org.apache.spark.util.{AccumulatorV2, Utils} -// Task result. Also contains updates to accumulator variables. +// Task result. Also contains updates to accumulator variables and executor metric peaks. private[spark] sealed trait TaskResult[T] /** A reference to a DirectTaskResult that has been stored in the worker's BlockManager. */ private[spark] case class IndirectTaskResult[T](blockId: BlockId, size: Int) extends TaskResult[T] with Serializable -/** A TaskResult that contains the task's return value and accumulator updates. */ +/** A TaskResult that contains the task's return value, accumulator updates and metric peaks. */ private[spark] class DirectTaskResult[T]( var valueBytes: ByteBuffer, - var accumUpdates: Seq[AccumulatorV2[_, _]]) + var accumUpdates: Seq[AccumulatorV2[_, _]], + var metricPeaks: Array[Long]) extends TaskResult[T] with Externalizable { private var valueObjectDeserialized = false private var valueObject: T = _ - def this() = this(null.asInstanceOf[ByteBuffer], null) + def this() = this(null.asInstanceOf[ByteBuffer], null, + new Array[Long](ExecutorMetricType.numMetrics)) override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeInt(valueBytes.remaining) Utils.writeByteBuffer(valueBytes, out) out.writeInt(accumUpdates.size) accumUpdates.foreach(out.writeObject) + out.writeInt(metricPeaks.length) + metricPeaks.foreach(out.writeLong) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -68,6 +73,17 @@ private[spark] class DirectTaskResult[T]( } accumUpdates = _accumUpdates } + + val numMetrics = in.readInt + if (numMetrics == 0) { + metricPeaks = Array.empty + } else { + val _metricPeaks = new ArrayBuffer[Long] + for (i <- 0 until numMetrics) { + _metricPeaks += in.readLong + } + metricPeaks = _metricPeaks.toArray + } valueObjectDeserialized = false } 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 bfdbf0217210..30c8f405317b 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,8 @@ package org.apache.spark.scheduler +import scala.collection.Map + import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId @@ -87,7 +89,7 @@ private[spark] trait TaskScheduler { execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorUpdates: ExecutorMetrics): Boolean + executorUpdates: Map[(Int, Int), 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 242486cc9bee..a413294a1e59 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -22,6 +22,7 @@ import java.util.{Locale, Timer, TimerTask} import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.AtomicLong +import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap, HashSet} import scala.util.Random @@ -628,7 +629,7 @@ private[spark] class TaskSchedulerImpl( execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: ExecutorMetrics): Boolean = { + executorUpdates: mutable.Map[(Int, Int), ExecutorMetrics]): Boolean = { // (taskId, stageId, stageAttemptId, accumUpdates) val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = { accumUpdates.flatMap { case (id, updates) => @@ -639,7 +640,7 @@ private[spark] class TaskSchedulerImpl( } } dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId, - executorMetrics) + executorUpdates) } def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index e7645fc19b9f..c8a7c729d043 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -827,7 +827,8 @@ private[spark] class TaskSetManager( // "result.value()" in "TaskResultGetter.enqueueSuccessfulTask" before reaching here. // Note: "result.value()" only deserializes the value when it's called at the first time, so // here "result.value()" just returns the value and won't block other threads. - sched.dagScheduler.taskEnded(tasks(index), Success, result.value(), result.accumUpdates, info) + sched.dagScheduler.taskEnded(tasks(index), Success, result.value(), result.accumUpdates, + result.metricPeaks, info) maybeFinishTaskSet() } @@ -858,6 +859,7 @@ private[spark] class TaskSetManager( val index = info.index copiesRunning(index) -= 1 var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty + val metricPeaks: Array[Long] = Array.empty val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}," + s" executor ${info.executorId}): ${reason.toErrorString}" val failureException: Option[Throwable] = reason match { @@ -934,7 +936,7 @@ private[spark] class TaskSetManager( isZombie = true } - sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info) + sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, metricPeaks, info) if (!isZombie && reason.countTowardsTaskFailures) { assert (null != failureReason) @@ -1018,7 +1020,7 @@ private[spark] class TaskSetManager( // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. sched.dagScheduler.taskEnded( - tasks(index), Resubmitted, null, Seq.empty, info) + tasks(index), Resubmitted, null, Seq.empty, Array.empty, info) } } } 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 0052fd42d276..e340b378e01e 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -836,9 +836,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 => + event.executorUpdates.foreach { case (_, peakUpdates) => liveExecutors.get(event.execId).foreach { exec => - if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(updates)) { + if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(peakUpdates)) { 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 bab1d5f0ee7c..b8ca4ee1a658 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -138,6 +138,7 @@ private[spark] object JsonProtocol { def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { val taskEndReason = taskEndReasonToJson(taskEnd.reason) val taskInfo = taskEnd.taskInfo + val executorMetrics = taskEnd.taskExecutorMetrics val taskMetrics = taskEnd.taskMetrics val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.taskEnd) ~ @@ -146,6 +147,7 @@ private[spark] object JsonProtocol { ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ ("Task Info" -> taskInfoToJson(taskInfo)) ~ + ("Task Executor Metrics" -> executorMetricsToJson(executorMetrics)) ~ ("Task Metrics" -> taskMetricsJson) } @@ -242,7 +244,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 executorUpdates = metricsUpdate.executorUpdates ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.metricsUpdate) ~ ("Executor ID" -> execId) ~ ("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) => @@ -251,7 +253,12 @@ private[spark] object JsonProtocol { ("Stage Attempt ID" -> stageAttemptId) ~ ("Accumulator Updates" -> JArray(updates.map(accumulableInfoToJson).toList)) }) ~ - ("Executor Metrics Updated" -> executorMetrics) + ("Executor Metrics Updated" -> executorUpdates.map { + case ((stageId, stageAttemptId), metrics) => + ("Stage ID" -> stageId) ~ + ("Stage Attempt ID" -> stageAttemptId) ~ + ("Executor Metrics" -> executorMetricsToJson(metrics)) + }) } def stageExecutorMetricsToJson(metrics: SparkListenerStageExecutorMetrics): JValue = { @@ -628,8 +635,10 @@ private[spark] object JsonProtocol { val taskType = (json \ "Task Type").extract[String] val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") val taskInfo = taskInfoFromJson(json \ "Task Info") + val executorMetrics = executorMetricsFromJson(json \ "Task Executor Metrics") val taskMetrics = taskMetricsFromJson(json \ "Task Metrics") - SparkListenerTaskEnd(stageId, stageAttemptId, taskType, taskEndReason, taskInfo, taskMetrics) + SparkListenerTaskEnd(stageId, stageAttemptId, taskType, taskEndReason, taskInfo, + executorMetrics, taskMetrics) } def jobStartFromJson(json: JValue): SparkListenerJobStart = { @@ -733,8 +742,14 @@ private[spark] object JsonProtocol { (json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson) (taskId, stageId, stageAttemptId, updates) } - val executorUpdates = jsonOption(json \ "Executor Metrics Updated").map { - executorUpdate => executorMetricsFromJson(executorUpdate) + val executorUpdates = (json \ "Executor Metrics Updated") match { + case JNothing => Map.empty[(Int, Int), ExecutorMetrics] + case value: JValue => value.extract[List[JValue]].map { json => + val stageId = (json \ "Stage ID").extract[Int] + val stageAttemptId = (json \ "Stage Attempt ID").extract[Int] + val executorMetrics = executorMetricsFromJson(json \ "Executor Metrics") + ((stageId, stageAttemptId) -> executorMetrics) + }.toMap } SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates) } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 191b516661e4..00a9e7fe21dc 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -23,6 +23,7 @@ import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{mock, never, verify, when} import org.scalatest.PrivateMethodTester +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.MetricsSystem @@ -283,18 +284,18 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // submit another attempt for the stage. We count completions from the first zombie attempt val stageAttempt1 = createStageInfo(stage.stageId, 5, attemptId = 1) post(SparkListenerStageSubmitted(stageAttempt1)) - post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo1, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo1, new ExecutorMetrics, null)) assert(totalRunningTasks(manager) === 1) val attemptTaskInfo1 = createTaskInfo(3, 0, "executor-1") val attemptTaskInfo2 = createTaskInfo(4, 1, "executor-1") post(SparkListenerTaskStart(0, 1, attemptTaskInfo1)) post(SparkListenerTaskStart(0, 1, attemptTaskInfo2)) assert(totalRunningTasks(manager) === 3) - post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo1, null)) + post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo1, new ExecutorMetrics, null)) assert(totalRunningTasks(manager) === 2) - post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo2, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo2, new ExecutorMetrics, null)) assert(totalRunningTasks(manager) === 1) - post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo2, null)) + post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo2, new ExecutorMetrics, null)) assert(totalRunningTasks(manager) === 0) } @@ -320,10 +321,10 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerTaskStart(2, 0, task2Info)) task1Info.markFinished(TaskState.FINISHED, System.currentTimeMillis()) - post(SparkListenerTaskEnd(2, 0, null, Success, task1Info, null)) + post(SparkListenerTaskEnd(2, 0, null, Success, task1Info, new ExecutorMetrics, null)) task2Info.markFinished(TaskState.FINISHED, System.currentTimeMillis()) - post(SparkListenerTaskEnd(2, 0, null, Success, task2Info, null)) + post(SparkListenerTaskEnd(2, 0, null, Success, task2Info, new ExecutorMetrics, null)) assert(adjustRequestedExecutors(manager) === -1) } @@ -439,7 +440,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // Remove executors when numExecutorsTarget is lower than current number of executors (1 to 3).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => - post(SparkListenerTaskEnd(0, 0, null, Success, info, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) } adjustRequestedExecutors(manager) assert(manager.executorMonitor.executorCount === 8) @@ -452,7 +453,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { onExecutorRemoved(manager, "3") // numExecutorsTarget is lower than minNumExecutors - post(SparkListenerTaskEnd(0, 0, null, Success, createTaskInfo(4, 4, "4"), null)) + post(SparkListenerTaskEnd(0, 0, null, Success, createTaskInfo(4, 4, "4"), + new ExecutorMetrics, null)) assert(manager.executorMonitor.executorCount === 5) assert(numExecutorsTarget(manager) === 5) assert(maxNumExecutorsNeeded(manager) == 4) @@ -832,7 +834,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // If the task is failed, we expect it to be resubmitted later. val taskEndReason = ExceptionFailure(null, null, null, null, None) - post(SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, null)) + post(SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, new ExecutorMetrics, null)) assert(maxNumExecutorsNeeded(manager) === 1) } @@ -946,7 +948,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // have one task finish -- we should adjust the target number of executors down // but we should *not* kill any executors yet - post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo0, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo0, new ExecutorMetrics, null)) assert(maxNumExecutorsNeeded(manager) === 1) assert(numExecutorsTarget(manager) === 2) clock.advance(1000) diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index dc4f4b4c66d9..bb092a1251d5 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -215,8 +215,9 @@ class HeartbeatReceiverSuite executorShouldReregister: Boolean): Unit = { val metrics = TaskMetrics.empty val blockManagerId = BlockManagerId(executorId, "localhost", 12345) - val executorUpdates = new ExecutorMetrics(Array(123456L, 543L, 12345L, 1234L, 123L, + val executorMetrics = new ExecutorMetrics(Array(123456L, 543L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) + val executorUpdates = Map((0, 0) -> executorMetrics) 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 d58ee4e651e1..0d5fe82651c7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -144,7 +144,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorUpdates: ExecutorMetrics): Boolean = true + executorUpdates: scala.collection.Map[(Int, Int), ExecutorMetrics]): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -676,7 +676,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: ExecutorMetrics): Boolean = true + executorMetrics: scala.collection.Map[(Int, Int), 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 @@ -1313,10 +1313,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // complete two tasks runEvent(makeCompletionEvent( taskSets(0).tasks(0), Success, 42, - Seq.empty, createFakeTaskInfoWithId(0))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(0))) runEvent(makeCompletionEvent( taskSets(0).tasks(1), Success, 42, - Seq.empty, createFakeTaskInfoWithId(1))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(1))) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) // verify stage exists assert(scheduler.stageIdToStage.contains(0)) @@ -1325,10 +1325,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // finish other 2 tasks runEvent(makeCompletionEvent( taskSets(0).tasks(2), Success, 42, - Seq.empty, createFakeTaskInfoWithId(2))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(2))) runEvent(makeCompletionEvent( taskSets(0).tasks(3), Success, 42, - Seq.empty, createFakeTaskInfoWithId(3))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(3))) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(endedTasks.size == 4) @@ -1339,14 +1339,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // with a speculative task and make sure the event is sent out runEvent(makeCompletionEvent( taskSets(0).tasks(3), Success, 42, - Seq.empty, createFakeTaskInfoWithId(5))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(5))) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(endedTasks.size == 5) // make sure non successful tasks also send out event runEvent(makeCompletionEvent( taskSets(0).tasks(3), UnknownReason, 42, - Seq.empty, createFakeTaskInfoWithId(6))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(6))) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) assert(endedTasks.size == 6) } @@ -2777,7 +2777,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Finish the first task of the result stage runEvent(makeCompletionEvent( taskSets.last.tasks(0), Success, 42, - Seq.empty, createFakeTaskInfoWithId(0))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(0))) // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( @@ -2820,7 +2820,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Finish the first task of the result stage runEvent(makeCompletionEvent( taskSets.last.tasks(0), Success, 42, - Seq.empty, createFakeTaskInfoWithId(0))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(0))) // Fail the second task with FetchFailed. runEvent(makeCompletionEvent( @@ -2965,6 +2965,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi reason: TaskEndReason, result: Any, extraAccumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty, + metricPeaks: Array[Long] = Array.empty, taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { val accumUpdates = reason match { case Success => task.metrics.accumulators() @@ -2972,7 +2973,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi case tk: TaskKilled => tk.accums case _ => Seq.empty } - CompletionEvent(task, reason, result, accumUpdates ++ extraAccumUpdates, taskInfo) + CompletionEvent(task, reason, result, accumUpdates ++ extraAccumUpdates, metricPeaks, taskInfo) } } 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 5b8d254e9ea4..f7de8745ca74 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -297,66 +297,66 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit createStageSubmittedEvent(0), // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor - createExecutorMetricsUpdateEvent(1, + createExecutorMetricsUpdateEvent(List(0), 1, new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, 6500L, 2500L, 5500L, 1500L, 10L, 90L, 2L, 20L))), - createExecutorMetricsUpdateEvent(2, + createExecutorMetricsUpdateEvent(List(0), 2, new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))), // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 - createExecutorMetricsUpdateEvent(1, + createExecutorMetricsUpdateEvent(List(0), 1, new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 - createExecutorMetricsUpdateEvent(2, + createExecutorMetricsUpdateEvent(List(0), 2, new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L))), // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 - createExecutorMetricsUpdateEvent(1, + createExecutorMetricsUpdateEvent(List(0), 1, new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, 7000L, 2500L, 6000L, 1500L, 10L, 90L, 2L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 - createExecutorMetricsUpdateEvent(2, + createExecutorMetricsUpdateEvent(List(0), 2, new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks - createExecutorMetricsUpdateEvent(1, + createExecutorMetricsUpdateEvent(List(0, 1), 1, new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 3, 6, 7, 9; // initialize stage 1 peaks - createExecutorMetricsUpdateEvent(2, + createExecutorMetricsUpdateEvent(List(0, 1), 2, new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 - createExecutorMetricsUpdateEvent(1, + createExecutorMetricsUpdateEvent(List(1), 1, new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))), // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 - createExecutorMetricsUpdateEvent(2, + createExecutorMetricsUpdateEvent(List(1), 2, new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L))), // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 - createExecutorMetricsUpdateEvent(1, + createExecutorMetricsUpdateEvent(List(1), 1, new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, 2000L, 1500L, 1000L, 500L, 10L, 90L, 2L, 20L))), // exec 2: new stage 1 peak for metrics at index: 7 - createExecutorMetricsUpdateEvent(2, + createExecutorMetricsUpdateEvent(List(1), 2, new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L, 10L, 90L, 2L, 20L))), // exec 1: no new stage 1 peaks - createExecutorMetricsUpdateEvent(1, + createExecutorMetricsUpdateEvent(List(1), 1, new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L, 10L, 90L, 2L, 20L))), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 - createExecutorMetricsUpdateEvent(2, + createExecutorMetricsUpdateEvent(List(1), 2, new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, 4000L, 6000L, 3000L, 5000L, 2000L, 10L, 90L, 2L, 20L))), createStageCompletedEvent(1), @@ -445,13 +445,15 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } private def createExecutorMetricsUpdateEvent( + stageIds: Seq[Int], 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)) + val executorUpdates = stageIds.map(id => (id, 0) -> executorMetrics).toMap + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorUpdates) } /** Check that the Spark history log line matches the expected event. */ 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 ead34e535723..f38dd2cba025 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.Map + import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -94,5 +96,5 @@ private class DummyTaskScheduler extends TaskScheduler { execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: ExecutorMetrics): Boolean = true + executorMetrics: Map[(Int, Int), 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 7d0712b353d6..e79613749f0c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -218,9 +218,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).isInstanceOf[SparkListenerStageExecutorMetrics] - } + val originalEvents = sc.eventLogger.get.loggedEvents 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/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 577d77e890d7..96706536fe53 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -311,7 +311,7 @@ private[spark] abstract class MockBackend( def taskSuccess(task: TaskDescription, result: Any): Unit = { val ser = env.serializer.newInstance() val resultBytes = ser.serialize(result) - val directResult = new DirectTaskResult(resultBytes, Seq()) // no accumulator updates + val directResult = new DirectTaskResult(resultBytes, Seq(), Array()) // no accumulator updates taskUpdate(task, TaskState.FINISHED, directResult) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index d1b16163a9f0..ff04239a7586 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -431,11 +431,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } // End the other task of the taskset, doesn't matter whether it succeeds or fails. val otherTask = tasks(1) - val result = new DirectTaskResult[Int](valueSer.serialize(otherTask.taskId), Seq()) + val result = new DirectTaskResult[Int](valueSer.serialize(otherTask.taskId), Seq(), Array()) tsm.handleSuccessfulTask(otherTask.taskId, result) } else { tasks.foreach { task => - val result = new DirectTaskResult[Int](valueSer.serialize(task.taskId), Seq()) + val result = new DirectTaskResult[Int](valueSer.serialize(task.taskId), Seq(), Array()) tsm.handleSuccessfulTask(task.taskId, result) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index da566dd82bce..bc3578a4f428 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -47,6 +47,7 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) reason: TaskEndReason, result: Any, accumUpdates: Seq[AccumulatorV2[_, _]], + metricPeaks: Array[Long], taskInfo: TaskInfo) { taskScheduler.endedTasks(taskInfo.index) = reason } @@ -716,8 +717,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg reason: TaskEndReason, result: Any, accumUpdates: Seq[AccumulatorV2[_, _]], + metricPeaks: Array[Long], taskInfo: TaskInfo): Unit = { - super.taskEnded(task, reason, result, accumUpdates, taskInfo) + super.taskEnded(task, reason, result, accumUpdates, metricPeaks, taskInfo) reason match { case Resubmitted => resubmittedTasks += 1 case _ => @@ -745,7 +747,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.runningTasks === 2) assert(manager.isZombie === false) - val directTaskResult = new DirectTaskResult[String](null, Seq()) { + val directTaskResult = new DirectTaskResult[String](null, Seq(), Array()) { override def value(resultSer: SerializerInstance): String = "" } // Complete one copy of the task, which should result in the task set manager @@ -797,8 +799,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg reason: TaskEndReason, result: Any, accumUpdates: Seq[AccumulatorV2[_, _]], + metricPeaks: Array[Long], taskInfo: TaskInfo): Unit = { - super.taskEnded(task, reason, result, accumUpdates, taskInfo) + super.taskEnded(task, reason, result, accumUpdates, metricPeaks, taskInfo) reason match { case Resubmitted => resubmittedTasks += 1 case _ => @@ -1204,7 +1207,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched.dagScheduler = mockDAGScheduler val taskSet = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1)) - when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any())).thenAnswer( + when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any(), any())).thenAnswer( (invocationOnMock: InvocationOnMock) => assert(manager.isZombie)) val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) @@ -1439,8 +1442,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg reason: TaskEndReason, result: Any, accumUpdates: Seq[AccumulatorV2[_, _]], + metricPeaks: Array[Long], taskInfo: TaskInfo): Unit = { - super.taskEnded(task, reason, result, accumUpdates, taskInfo) + super.taskEnded(task, reason, result, accumUpdates, metricPeaks, taskInfo) reason match { case Resubmitted => resubmittedTasks += taskInfo.index case _ => @@ -1515,9 +1519,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg private def createTaskResult( id: Int, - accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { + accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty, + metricPeaks: Array[Long] = Array.empty): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() - new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates) + new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates, metricPeaks) } test("SPARK-13343 speculative tasks that didn't commit shouldn't be marked as success") { @@ -1585,9 +1590,10 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg TaskKilled("Finish but did not commit due to another attempt succeeded"), null, Seq.empty, + Array.empty, info4) verify(sched.dagScheduler).taskEnded(manager.tasks(3), Success, result.value(), - result.accumUpdates, info3) + result.accumUpdates, Array.empty, info3) } test("SPARK-13704 Rack Resolution is done with a batch of de-duped hosts") { 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 b5800661efa7..7d73546ae227 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -301,7 +301,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { time += 1 s1Tasks.head.markFinished(TaskState.FAILED, time) listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber, - "taskType", TaskResultLost, s1Tasks.head, null)) + "taskType", TaskResultLost, s1Tasks.head, new ExecutorMetrics, null)) time += 1 val reattempt = newAttempt(s1Tasks.head, nextTaskId()) @@ -336,7 +336,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { killed.finishTime = time killed.failed = true listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber, - "taskType", TaskKilled("killed"), killed, null)) + "taskType", TaskKilled("killed"), killed, new ExecutorMetrics, null)) check[JobDataWrapper](1) { job => assert(job.info.numKilledTasks === 1) @@ -364,7 +364,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { denied.finishTime = time denied.failed = true listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber, - "taskType", denyReason, denied, null)) + "taskType", denyReason, denied, new ExecutorMetrics, null)) check[JobDataWrapper](1) { job => assert(job.info.numKilledTasks === 2) @@ -397,7 +397,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { pending.foreach { task => task.markFinished(TaskState.FINISHED, time) listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptNumber, - "taskType", Success, task, s1Metrics)) + "taskType", Success, task, new ExecutorMetrics, s1Metrics)) } check[JobDataWrapper](1) { job => @@ -486,7 +486,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { s2Tasks.foreach { task => task.markFinished(TaskState.FAILED, time) listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptNumber, - "taskType", TaskResultLost, task, null)) + "taskType", TaskResultLost, task, new ExecutorMetrics, null)) } check[JobDataWrapper](1) { job => @@ -538,7 +538,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { newS2Tasks.foreach { task => task.markFinished(TaskState.FINISHED, time) listener.onTaskEnd(SparkListenerTaskEnd(newS2.stageId, newS2.attemptNumber, "taskType", - Success, task, null)) + Success, task, new ExecutorMetrics, null)) } time += 1 @@ -596,7 +596,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { j2s2Tasks.foreach { task => task.markFinished(TaskState.FINISHED, time) listener.onTaskEnd(SparkListenerTaskEnd(j2Stages.last.stageId, j2Stages.last.attemptNumber, - "taskType", Success, task, null)) + "taskType", Success, task, new ExecutorMetrics, null)) } time += 1 @@ -1016,7 +1016,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val metrics = TaskMetrics.empty metrics.setExecutorRunTime(42L) listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptNumber, - "taskType", Success, task, metrics)) + "taskType", Success, task, new ExecutorMetrics, metrics)) new AppStatusStore(store) .taskSummary(dropped.stageId, dropped.attemptNumber, Array(0.25d, 0.50d, 0.75d)) @@ -1189,11 +1189,13 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { time += 1 tasks(1).markFinished(TaskState.FINISHED, time) listener.onTaskEnd(SparkListenerTaskEnd( - stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(1), null)) + stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(1), + new ExecutorMetrics, null)) time += 1 tasks(0).markFinished(TaskState.FINISHED, time) listener.onTaskEnd(SparkListenerTaskEnd( - stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(0), null)) + stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(0), + new ExecutorMetrics, null)) // Start task 3 and task 2 should be evicted. listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, tasks(2))) @@ -1259,7 +1261,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { time += 1 tasks(0).markFinished(TaskState.FINISHED, time) listener.onTaskEnd(SparkListenerTaskEnd( - stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(0), null)) + stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(0), + new ExecutorMetrics, null)) // Stage 1 Completed stage1.failureReason = Some("Failed") @@ -1274,7 +1277,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { tasks(1).markFinished(TaskState.FINISHED, time) listener.onTaskEnd( SparkListenerTaskEnd(stage1.stageId, stage1.attemptNumber, "taskType", - TaskKilled(reason = "Killed"), tasks(1), null)) + TaskKilled(reason = "Killed"), tasks(1), new ExecutorMetrics, null)) // Ensure killed task metrics are updated val allStages = store.view(classOf[StageDataWrapper]).reverse().asScala.map(_.info) @@ -1316,11 +1319,11 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { time += 1 tasks(0).markFinished(TaskState.FINISHED, time) listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", - Success, tasks(0), null)) + Success, tasks(0), new ExecutorMetrics, null)) time += 1 tasks(1).markFinished(TaskState.FINISHED, time) listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", - Success, tasks(1), null)) + Success, tasks(1), new ExecutorMetrics, null)) stage.failureReason = Some("Failed") listener.onStageCompleted(SparkListenerStageCompleted(stage)) @@ -1331,11 +1334,13 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { time += 1 tasks(2).markFinished(TaskState.FAILED, time) listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", - ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), null)) + ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), new ExecutorMetrics, + null)) time += 1 tasks(3).markFinished(TaskState.FAILED, time) listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", - ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), null)) + ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), new ExecutorMetrics, + null)) val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info) esummary.foreach { execSummary => @@ -1384,65 +1389,65 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(0, 1, Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, 6500L, 2500L, 5500L, 1500L))) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(0, 2, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(0, 1, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(0, 2, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(0, 1, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(0, 2, 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, 3, 7 - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(0, 1, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(0, 2, 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, 3 - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, 1, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, 2, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, 1, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, 2, 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, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, 1, 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: 6 - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, 2, Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, 4000L, 6000L, 3000L, 5000L, 2000L))) listener.onStageCompleted(createStageCompletedEvent(1)) @@ -1682,13 +1687,14 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { /** Create an executor metrics update event, with the specified executor metrics values. */ private def createExecutorMetricsUpdateEvent( + stageId: Int, executorId: Int, executorMetrics: Array[Long]): 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(new ExecutorMetrics(executorMetrics))) + val executorUpdates = Map((stageId, 0) -> new ExecutorMetrics(executorMetrics)) + SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorUpdates) } } diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index f4896983d565..bd18e9e628da 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -25,7 +25,7 @@ import scala.xml.Node import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.apache.spark._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.Status._ import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore @@ -141,8 +141,10 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis()) val taskMetrics = TaskMetrics.empty + val executorMetrics = new ExecutorMetrics taskMetrics.incPeakExecutionMemory(peakExecutionMemory) - listener.onTaskEnd(SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, + executorMetrics, taskMetrics)) } listener.onStageCompleted(SparkListenerStageCompleted(stageInfo)) page.render(request) 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 f2f62d6d4df1..f4673dc51ee9 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -49,12 +49,18 @@ class JsonProtocolSuite extends SparkFunSuite { SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 5, 3000L, true)) val taskEnd = SparkListenerTaskEnd(1, 0, "ShuffleMapTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), + new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = false, hasOutput = false)) val taskEndWithHadoopInput = SparkListenerTaskEnd(1, 0, "ShuffleMapTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), + new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = false)) val taskEndWithOutput = SparkListenerTaskEnd(1, 0, "ResultTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), + new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)) val jobStart = { val stageIds = Seq[Int](1, 2, 3, 4) @@ -101,7 +107,7 @@ class JsonProtocolSuite extends SparkFunSuite { Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L, 10L, 90L, 2L, 20L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), - Some(executorUpdates)) + Map((0, 0) -> executorUpdates)) } val blockUpdated = SparkListenerBlockUpdated(BlockUpdatedInfo(BlockManagerId("Stars", @@ -437,8 +443,8 @@ class JsonProtocolSuite extends SparkFunSuite { val oldExecutorMetricsUpdateJson = JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) .removeField( _._1 == "Executor Metrics Updated") - val exepectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", true, false) - assertEquals(exepectedExecutorMetricsUpdate, + val expectedExecutorMetricsUpdate = makeExecutorMetricsUpdate("1", true, false) + assertEquals(expectedExecutorMetricsUpdate, JsonProtocol.executorMetricsUpdateFromJson(oldExecutorMetricsUpdateJson)) } @@ -556,6 +562,9 @@ private[spark] object JsonProtocolSuite extends Assertions { * --------------------------------- */ private[spark] def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) { + def lexOrder(x: (Int, Int), y: (Int, Int)) = + x._1 < y._1 || x._2 < y._2 + (event1, event2) match { case (e1: SparkListenerStageSubmitted, e2: SparkListenerStageSubmitted) => assert(e1.properties === e2.properties) @@ -569,9 +578,11 @@ private[spark] object JsonProtocolSuite extends Assertions { assertEquals(e1.taskInfo, e2.taskInfo) case (e1: SparkListenerTaskEnd, e2: SparkListenerTaskEnd) => assert(e1.stageId === e2.stageId) + assert(e1.stageAttemptId === e2.stageAttemptId) assert(e1.taskType === e2.taskType) assertEquals(e1.reason, e2.reason) assertEquals(e1.taskInfo, e2.taskInfo) + assertEquals(e1.taskExecutorMetrics, e2.taskExecutorMetrics) assertEquals(e1.taskMetrics, e2.taskMetrics) case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => assert(e1.jobId === e2.jobId) @@ -600,8 +611,16 @@ private[spark] object JsonProtocolSuite extends Assertions { assert(stageAttemptId1 === stageAttemptId2) assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) }) - assertOptionEquals(e1.executorUpdates, e2.executorUpdates, - (e1: ExecutorMetrics, e2: ExecutorMetrics) => assertEquals(e1, e2)) + assertSeqEquals[((Int, Int), ExecutorMetrics)]( + e1.executorUpdates.toSeq.sortWith((x, y) => lexOrder(x._1, y._1)), + e2.executorUpdates.toSeq.sortWith((x, y) => lexOrder(x._1, y._1)), + (a, b) => { + val (k1, v1) = a + val (k2, v2) = b + assert(k1 === k2) + assertEquals(v1, v2) + } + ) case (e1: SparkListenerStageExecutorMetrics, e2: SparkListenerStageExecutorMetrics) => assert(e1.execId === e2.execId) assert(e1.stageId === e2.stageId) @@ -885,12 +904,12 @@ private[spark] object JsonProtocolSuite extends Assertions { } else { Seq() } - val executorMetricsUpdate = + val executorMetricsUpdate: Map[(Int, Int), ExecutorMetrics] = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, + Map((0, 0) -> new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L, 10L, 90L, 2L, 20L))) } else { - None + Map.empty } SparkListenerExecutorMetricsUpdate(execId, taskMetrics, executorMetricsUpdate) } @@ -1193,6 +1212,24 @@ private[spark] object JsonProtocolSuite extends Assertions { | } | ] | }, + | "Task Executor Metrics" : { + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 + | }, | "Task Metrics": { | "Executor Deserialize Time": 300, | "Executor Deserialize CPU Time": 300, @@ -1294,6 +1331,24 @@ private[spark] object JsonProtocolSuite extends Assertions { | } | ] | }, + | "Task Executor Metrics" : { + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 + | }, | "Task Metrics": { | "Executor Deserialize Time": 300, | "Executor Deserialize CPU Time": 300, @@ -1395,6 +1450,24 @@ private[spark] object JsonProtocolSuite extends Assertions { | } | ] | }, + | "Task Executor Metrics" : { + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 + | }, | "Task Metrics": { | "Executor Deserialize Time": 300, | "Executor Deserialize CPU Time": 300, @@ -2090,29 +2163,34 @@ private[spark] object JsonProtocolSuite extends Assertions { | ] | } | ], - | "Executor Metrics Updated" : { - | "JVMHeapMemory" : 543, - | "JVMOffHeapMemory" : 123456, - | "OnHeapExecutionMemory" : 12345, - | "OffHeapExecutionMemory" : 1234, - | "OnHeapStorageMemory" : 123, - | "OffHeapStorageMemory" : 12, - | "OnHeapUnifiedMemory" : 432, - | "OffHeapUnifiedMemory" : 321, - | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765, - | "ProcessTreeJVMVMemory": 256912, - | "ProcessTreeJVMRSSMemory": 123456, - | "ProcessTreePythonVMemory": 123456, - | "ProcessTreePythonRSSMemory": 61728, - | "ProcessTreeOtherVMemory": 30364, - | "ProcessTreeOtherRSSMemory": 15182, - | "MinorGCCount": 10, - | "MinorGCTime": 90, - | "MajorGCCount": 2, - | "MajorGCTime": 20 - | } - | + | "Executor Metrics Updated" : [ + | { + | "Stage ID" : 0, + | "Stage Attempt ID" : 0, + | "Executor Metrics" : { + | "JVMHeapMemory" : 543, + | "JVMOffHeapMemory" : 123456, + | "OnHeapExecutionMemory" : 12345, + | "OffHeapExecutionMemory" : 1234, + | "OnHeapStorageMemory" : 123, + | "OffHeapStorageMemory" : 12, + | "OnHeapUnifiedMemory" : 432, + | "OffHeapUnifiedMemory" : 321, + | "DirectPoolMemory" : 654, + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182, + | "MinorGCCount": 10, + | "MinorGCTime": 90, + | "MajorGCCount": 2, + | "MajorGCTime": 20 + | } + | } + | ] |} """.stripMargin From 03e41a83023ff2bf1681888f68e0b5d177ab013d Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Tue, 12 Feb 2019 15:53:16 -0800 Subject: [PATCH 02/22] [SPARK-26329][CORE] Fix test compilation error in sql. --- .../spark/sql/execution/ui/SQLAppStatusListenerSuite.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 8edbb8770671..bb3f6d8236fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Status._ import org.apache.spark.rdd.RDD @@ -232,6 +233,7 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with taskType = "", reason = null, createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 100)), + new ExecutorMetrics, null)) checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 2)) @@ -243,6 +245,7 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with taskType = "", reason = null, createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 2)), + new ExecutorMetrics, null)) listener.onTaskEnd(SparkListenerTaskEnd( stageId = 0, @@ -250,6 +253,7 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with taskType = "", reason = null, createTaskInfo(1, 0, accums = accumulatorUpdates.mapValues(_ * 3)), + new ExecutorMetrics, null)) checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 5)) @@ -272,6 +276,7 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with taskType = "", reason = null, createTaskInfo(0, 0, accums = accumulatorUpdates.mapValues(_ * 3)), + new ExecutorMetrics, null)) listener.onTaskEnd(SparkListenerTaskEnd( stageId = 1, @@ -279,6 +284,7 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with taskType = "", reason = null, createTaskInfo(1, 0, accums = accumulatorUpdates.mapValues(_ * 3)), + new ExecutorMetrics, null)) checkAnswer(statusStore.executionMetrics(executionId), accumulatorUpdates.mapValues(_ * 11)) From 7f6bd74659701efb346c8466cfc845f43ff4bba1 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Tue, 12 Feb 2019 17:47:12 -0800 Subject: [PATCH 03/22] [SPARK-26329][CORE] Fix Mima issues. --- project/MimaExcludes.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 5978f88d6a46..90a6148dc53d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -73,6 +73,13 @@ object MimaExcludes { // [SPARK-26254][CORE] Extract Hive + Kafka dependencies from Core. ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.deploy.security.HiveDelegationTokenProvider"), + // [SPARK-26329][CORE] Faster polling of executor memory metrics. + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd$"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.apply"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.copy$default$6"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerTaskEnd.this"), + // [SPARK-26311][CORE]New feature: apply custom log URL pattern for executor log URLs ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.apply"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.copy"), From 73978975059855abb96b5fe444f0a3fd6f09bf9e Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Tue, 12 Feb 2019 22:41:15 -0800 Subject: [PATCH 04/22] [SPARK-26329][CORE] Fix possible NPE. --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 098ee291c431..2866f697ebb3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -658,7 +658,7 @@ private[spark] class Executor( } // If the count is zero, the stage is removed from activeStages - activeStages.compute((stageId, stageAttemptId), decrementCount) + activeStages.computeIfPresent((stageId, stageAttemptId), decrementCount) // If the stage has been removed from activeStages, remove it from stageMetricPeaks too def removeInactive(k: StageKey): Unit = { From e1aeafc3b0b64308f3185583dad219b176ffa43b Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Tue, 5 Mar 2019 20:56:52 -0800 Subject: [PATCH 05/22] [SPARK-26329][CORE] Fix JsonProtocolSuite post-rebase to account for SPARK-25865 SPARK-25865 added GC information to ExecutorMetrics. --- .../apache/spark/util/JsonProtocolSuite.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 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 f4673dc51ee9..43be0881d8b2 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1228,7 +1228,11 @@ private[spark] object JsonProtocolSuite extends Assertions { | "ProcessTreePythonVMemory": 123456, | "ProcessTreePythonRSSMemory": 61728, | "ProcessTreeOtherVMemory": 30364, - | "ProcessTreeOtherRSSMemory": 15182 + | "ProcessTreeOtherRSSMemory": 15182, + | "MinorGCCount" : 0, + | "MinorGCTime" : 0, + | "MajorGCCount" : 0, + | "MajorGCTime" : 0 | }, | "Task Metrics": { | "Executor Deserialize Time": 300, @@ -1347,7 +1351,11 @@ private[spark] object JsonProtocolSuite extends Assertions { | "ProcessTreePythonVMemory": 123456, | "ProcessTreePythonRSSMemory": 61728, | "ProcessTreeOtherVMemory": 30364, - | "ProcessTreeOtherRSSMemory": 15182 + | "ProcessTreeOtherRSSMemory": 15182, + | "MinorGCCount" : 0, + | "MinorGCTime" : 0, + | "MajorGCCount" : 0, + | "MajorGCTime" : 0 | }, | "Task Metrics": { | "Executor Deserialize Time": 300, @@ -1466,7 +1474,11 @@ private[spark] object JsonProtocolSuite extends Assertions { | "ProcessTreePythonVMemory": 123456, | "ProcessTreePythonRSSMemory": 61728, | "ProcessTreeOtherVMemory": 30364, - | "ProcessTreeOtherRSSMemory": 15182 + | "ProcessTreeOtherRSSMemory": 15182, + | "MinorGCCount" : 0, + | "MinorGCTime" : 0, + | "MajorGCCount" : 0, + | "MajorGCTime" : 0 | }, | "Task Metrics": { | "Executor Deserialize Time": 300, From 75ba39d54bfe2901330f698f5fd0f338415138f3 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Fri, 8 Mar 2019 15:09:43 -0800 Subject: [PATCH 06/22] [SPARK-26329][CORE] Extract polling logic into a separate class. Address review feedback from Edwina Lu and Imran Rashid. Yet to address: handle sending task executor metrics on task failure. --- .../org/apache/spark/HeartbeatReceiver.scala | 6 +- .../org/apache/spark/executor/Executor.scala | 140 ++---------- .../executor/ExecutorMetricsPoller.scala | 210 ++++++++++++++++++ .../scheduler/EventLoggingListener.scala | 28 ++- 4 files changed, 247 insertions(+), 137 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 1f4ea56a8c2e..4e6679f5abcf 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -39,9 +39,11 @@ import org.apache.spark.util._ */ private[spark] case class Heartbeat( executorId: String, - accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates + // taskId -> accumulator updates + accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorUpdates: Map[(Int, Int), ExecutorMetrics]) // executor level updates + // (stageId, stageAttemptId) -> executor metric peaks + executorUpdates: Map[(Int, Int), ExecutorMetrics]) /** * An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is 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 2866f697ebb3..ff67dbe837cd 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -18,14 +18,12 @@ package org.apache.spark.executor import java.io.{File, NotSerializableException} -import java.lang.Long.{MAX_VALUE => LONG_MAX_VALUE} import java.lang.Thread.UncaughtExceptionHandler import java.lang.management.ManagementFactory import java.net.{URI, URL} import java.nio.ByteBuffer import java.util.Properties import java.util.concurrent._ -import java.util.concurrent.atomic.{AtomicLong, AtomicLongArray} import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ @@ -40,7 +38,6 @@ 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.metrics.ExecutorMetricType import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ @@ -173,17 +170,6 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] - type StageKey = (Int, Int) - - // Map of (stageId, stageAttemptId) to count of running tasks - private val activeStages = new ConcurrentHashMap[StageKey, AtomicLong] - - // Map of (stageId, stageAttemptId) to executor metric peaks - private val stageMetricPeaks = new ConcurrentHashMap[StageKey, AtomicLongArray] - - // Map of taskId to executor metric peaks - private val taskMetricPeaks = new ConcurrentHashMap[Long, AtomicLongArray] - /** * When an executor is unable to send heartbeats to the driver more than `HEARTBEAT_MAX_FAILURES` * times, it should kill itself. The default value is 60. For example, if max failures is 60 and @@ -207,36 +193,12 @@ private[spark] class Executor( */ private val METRICS_POLLING_INTERVAL_MS = conf.get(EXECUTOR_METRICS_POLLING_INTERVAL) - // Executor for the metrics polling task - private val poller = - if (METRICS_POLLING_INTERVAL_MS > 0) { - ThreadUtils.newDaemonSingleThreadScheduledExecutor("executor-poller") - } else { - null - } - - private def poll(): Unit = { - // get the latest values for the metrics - val latestMetrics = ExecutorMetrics.getCurrentMetrics(env.memoryManager) - - def compareAndUpdate(current: Long, latest: Long): Long = - if (latest > current) latest else current - - def updatePeaks(metrics: AtomicLongArray): Unit = { - (0 until metrics.length).foreach { i => - metrics.getAndAccumulate(i, latestMetrics(i), compareAndUpdate) - } - } - - def peaksForStage(k: StageKey, v: AtomicLong): AtomicLongArray = - if (v.get() > 0) stageMetricPeaks.get(k) else null + private val pollOnHeartbeat = if (METRICS_POLLING_INTERVAL_MS > 0) false else true - // for each active stage (number of running tasks > 0), get the peaks and update them - activeStages.forEach[AtomicLongArray](LONG_MAX_VALUE, peaksForStage, updatePeaks) - - // for each running task, update the peaks - taskMetricPeaks.forEachValue(LONG_MAX_VALUE, updatePeaks) - } + // Poller for the memory metrics. + private val metricsPoller = new ExecutorMetricsPoller( + env.memoryManager, + METRICS_POLLING_INTERVAL_MS) // Executor for the heartbeat task. private val heartbeater = new Heartbeater( @@ -256,19 +218,14 @@ private[spark] class Executor( heartbeater.start() - if (poller != null) { - val pollingTask = new Runnable() { - override def run(): Unit = Utils.logUncaughtExceptions(poll()) - } - poller.scheduleAtFixedRate(pollingTask, 0L, METRICS_POLLING_INTERVAL_MS, TimeUnit.MILLISECONDS) - } + metricsPoller.start() private[executor] def numRunningTasks: Int = runningTasks.size() def launchTask(context: ExecutorBackend, taskDescription: TaskDescription): Unit = { val tr = new TaskRunner(context, taskDescription) runningTasks.put(taskDescription.taskId, tr) - taskMetricPeaks.put(taskDescription.taskId, new AtomicLongArray(ExecutorMetricType.numMetrics)) + metricsPoller.onTaskLaunch(taskDescription.taskId) threadPool.execute(tr) } @@ -311,14 +268,11 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() - if (poller != null) { - try { - poller.shutdown() - poller.awaitTermination(METRICS_POLLING_INTERVAL_MS, TimeUnit.MILLISECONDS) - } catch { - case NonFatal(e) => - logWarning("Unable to stop poller", e) - } + try { + metricsPoller.stop() + } catch { + case NonFatal(e) => + logWarning("Unable to stop executor metrics poller", e) } try { heartbeater.stop() @@ -480,18 +434,7 @@ private[spark] class Executor( val stageId = task.stageId val stageAttemptId = task.stageAttemptId - val count = new AtomicLong(0) - val old = activeStages.putIfAbsent((stageId, stageAttemptId), count) - val stageCount = - if (old != null) { - old.incrementAndGet() - } else { - logDebug(s"added ($stageId, $stageAttemptId) to activeStages") - count.incrementAndGet() - } - logDebug(s"activeStages: ($stageId, $stageAttemptId) -> $stageCount") - stageMetricPeaks.putIfAbsent((stageId, stageAttemptId), - new AtomicLongArray(ExecutorMetricType.numMetrics)) + metricsPoller.onTaskStart(stageId, stageAttemptId) // Run the actual task and measure its runtime. taskStartTimeNs = System.nanoTime() @@ -608,19 +551,11 @@ private[spark] class Executor( executorSource.METRIC_DISK_BYTES_SPILLED.inc(task.metrics.diskBytesSpilled) executorSource.METRIC_MEMORY_BYTES_SPILLED.inc(task.metrics.memoryBytesSpilled) - def getMetricPeaks(): Array[Long] = { - val currentPeaks = taskMetricPeaks.get(taskId) - val metricPeaks = new Array[Long](ExecutorMetricType.numMetrics) - ExecutorMetricType.metricToOffset.foreach { case (_, i) => - metricPeaks(i) = currentPeaks.get(i) - } - metricPeaks - } - // Note: accumulator updates must be collected after TaskMetrics is updated val accumUpdates = task.collectAccumulatorUpdates() + val metricPeaks = metricsPoller.getTaskMetricPeaks(taskId) // TODO: do not serialize value twice - val directResult = new DirectTaskResult(valueBytes, accumUpdates, getMetricPeaks) + val directResult = new DirectTaskResult(valueBytes, accumUpdates, metricPeaks) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit() @@ -646,28 +581,7 @@ private[spark] class Executor( } } - def decrementCount(stage: StageKey, count: AtomicLong): AtomicLong = { - val countValue = count.decrementAndGet() - if (countValue == 0L) { - logDebug(s"removing (${stage._1}, ${stage._2}) from activeStages") - null - } else { - logDebug(s"activeStages: (${stage._1}, ${stage._2}) -> " + countValue) - count - } - } - - // If the count is zero, the stage is removed from activeStages - activeStages.computeIfPresent((stageId, stageAttemptId), decrementCount) - - // If the stage has been removed from activeStages, remove it from stageMetricPeaks too - def removeInactive(k: StageKey): Unit = { - if (activeStages.get(k) == null) { - stageMetricPeaks.remove(k) - } - } - stageMetricPeaks.forEachKey(LONG_MAX_VALUE, removeInactive) - + metricsPoller.onTaskCompletion(stageId, stageAttemptId) executorSource.SUCCEEDED_TASKS.inc(1L) setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) @@ -743,7 +657,7 @@ private[spark] class Executor( } } finally { runningTasks.remove(taskId) - taskMetricPeaks.remove(taskId) + metricsPoller.onTaskCleanup(taskId) } } @@ -960,25 +874,11 @@ private[spark] class Executor( val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]() val curGCTime = computeTotalGcTime() - // if not polling in a separater poller, poll here - if (poller == null) { - poll() - } - - // build the executor level memory metrics - val executorUpdates = new HashMap[StageKey, ExecutorMetrics] - - def peaksForStage(k: StageKey, v: AtomicLong): (StageKey, AtomicLongArray) = - if (v.get() > 0) (k, stageMetricPeaks.get(k)) else null - - def addPeaks(nested: (StageKey, AtomicLongArray)): Unit = { - val (k, v) = nested - executorUpdates.put(k, new ExecutorMetrics(v)) - // at the same time, reset the peaks in stageMetricPeaks - stageMetricPeaks.put(k, new AtomicLongArray(ExecutorMetricType.numMetrics)) + if (pollOnHeartbeat) { + metricsPoller.poll() } - activeStages.forEach[(StageKey, AtomicLongArray)](LONG_MAX_VALUE, peaksForStage, addPeaks) + val executorUpdates = metricsPoller.getExecutorUpdates() for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala new file mode 100644 index 000000000000..07ff7735e7ec --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -0,0 +1,210 @@ +/* + * 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.lang.Long.{MAX_VALUE => LONG_MAX_VALUE} +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.concurrent.atomic.{AtomicLong, AtomicLongArray} + +import scala.collection.mutable.HashMap + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.Logging +import org.apache.spark.memory.MemoryManager +import org.apache.spark.metrics.ExecutorMetricType +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * :: DeveloperApi :: + * A class that polls executor metrics, and tracks their peaks per task and per stage. + * Each executor keeps an instance of this class. + * + * @param memoryManager the memory manager used by the executor. + * @param pollingInterval the polling interval in milliseconds. + */ +@DeveloperApi +private[spark] class ExecutorMetricsPoller( + memoryManager: MemoryManager, + pollingInterval: Long) + extends Logging { + + type StageKey = (Int, Int) + // tuple for Task Count and Metric Peaks + type TCMP = (AtomicLong, AtomicLongArray) + + // Map of (stageId, stageAttemptId) to (count of running tasks, executor metric peaks) + private val stageTCMP = new ConcurrentHashMap[StageKey, TCMP] + + // Map of taskId to executor metric peaks + private val taskMetricPeaks = new ConcurrentHashMap[Long, AtomicLongArray] + + private val poller = + if (pollingInterval > 0) { + ThreadUtils.newDaemonSingleThreadScheduledExecutor("executor-poller") + } else { + null + } + + /** + * Function to poll executor metrics. + * On start, if pollingInterval is positive, this is scheduled to run at that interval. + * Otherwise, this is called by the reportHeartBeat function defined in Executor and passed + * to its Heartbeater. + */ + def poll(): Unit = { + // Note: Task runner threads may update stageTCMP or read from taskMetricPeaks concurrently + // with this function via calls to methods of this class. + + // get the latest values for the metrics + val latestMetrics = ExecutorMetrics.getCurrentMetrics(memoryManager) + + def compareAndUpdate(current: Long, latest: Long): Long = + if (latest > current) latest else current + + def updatePeaks(metrics: AtomicLongArray): Unit = { + (0 until metrics.length).foreach { i => + metrics.getAndAccumulate(i, latestMetrics(i), compareAndUpdate) + } + } + + // for each active stage, update the peaks + stageTCMP.forEachValue(LONG_MAX_VALUE, v => updatePeaks(v._2)) + + // for each running task, update the peaks + taskMetricPeaks.forEachValue(LONG_MAX_VALUE, updatePeaks) + } + + /** Starts the polling thread. */ + def start(): Unit = { + if (poller != null) { + val pollingTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(poll()) + } + poller.scheduleAtFixedRate(pollingTask, 0L, pollingInterval, TimeUnit.MILLISECONDS) + } + } + + /** + * Called by Executor#launchTask. + * + * @param taskId the id of the task being launched. + */ + def onTaskLaunch(taskId: Long): Unit = { + taskMetricPeaks.put(taskId, new AtomicLongArray(ExecutorMetricType.numMetrics)) + } + + /** + * Called by TaskRunner#run. + * + * @param stageId the id of the stage the task belongs to. + * @param stageAttemptId the attempt number of the stage the task belongs to. + */ + def onTaskStart(stageId: Int, stageAttemptId: Int): Unit = { + // Put a new entry in stageTCMP for the stage if there isn't one already. + // Increment the task count. + val (count, _) = stageTCMP.computeIfAbsent((stageId, stageAttemptId), + _ => (new AtomicLong(0), new AtomicLongArray(ExecutorMetricType.numMetrics))) + val stageCount = count.incrementAndGet() + logDebug(s"stageTCMP: ($stageId, $stageAttemptId) -> $stageCount") + } + + /** + * Called by TaskRunner#run. + * + * @param stageId the id of the stage the task belongs to. + * @param stageAttemptId the attempt number of the stage the task belongs to. + */ + def onTaskCompletion(stageId: Int, stageAttemptId: Int): Unit = { + // Decrement the task count. + // Remove the entry from stageTCMP if the task count reaches zero + + def decrementCount(stage: StageKey, countAndPeaks: TCMP): TCMP = { + val count = countAndPeaks._1 + val countValue = count.decrementAndGet() + if (countValue == 0L) { + logDebug(s"removing (${stage._1}, ${stage._2}) from stageTCMP") + null + } else { + logDebug(s"stageTCMP: (${stage._1}, ${stage._2}) -> " + countValue) + countAndPeaks + } + } + + stageTCMP.computeIfPresent((stageId, stageAttemptId), decrementCount) + } + + /** + * Called by TaskRunner#run. + * + * @param taskId the id of the task that was run. + */ + def onTaskCleanup(taskId: Long): Unit = { + taskMetricPeaks.remove(taskId) + } + + /** + * Called by TaskRunner#run. + * + * @param taskId the id of the task that was run. + */ + def getTaskMetricPeaks(taskId: Long): Array[Long] = { + val currentPeaks = taskMetricPeaks.get(taskId) + val metricPeaks = new Array[Long](ExecutorMetricType.numMetrics) + ExecutorMetricType.metricToOffset.foreach { case (_, i) => + metricPeaks(i) = currentPeaks.get(i) + } + metricPeaks + } + + + /** + * Called by the reportHeartBeat function defined in Executor and passed to its Heartbeater. + * It resets the metric peaks in stageTCMP before returning the executor updates. + * Thus, the executor updates contains the per-stage metric peaks since the last heartbeat + * (the last time this method was called). + */ + def getExecutorUpdates(): HashMap[StageKey, ExecutorMetrics] = { + // build the executor level memory metrics + val executorUpdates = new HashMap[StageKey, ExecutorMetrics] + + def peaksForStage(k: StageKey, v: TCMP): (StageKey, AtomicLongArray) = + if (v._1.get() > 0) (k, v._2) else null + + def addPeaks(nested: (StageKey, AtomicLongArray)): Unit = { + val (k, v) = nested + executorUpdates.put(k, new ExecutorMetrics(v)) + } + + stageTCMP.forEach[(StageKey, AtomicLongArray)](LONG_MAX_VALUE, peaksForStage, addPeaks) + + // reset the peaks + def resetPeaks(k: StageKey, v: TCMP): TCMP = + (v._1, new AtomicLongArray(ExecutorMetricType.numMetrics)) + + stageTCMP.replaceAll(resetPeaks) + + executorUpdates + } + + /** Stops the polling thread. */ + def stop(): Unit = { + if (poller != null) { + poller.shutdown() + poller.awaitTermination(10, TimeUnit.SECONDS) + } + } +} 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 5f196bbe31b9..0f5aeb18f517 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -94,7 +94,7 @@ private[spark] class EventLoggingListener( // Visible for tests only. private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) - // map of (stageId, stageAttempt), to peak executor metrics for the stage + // map of (stageId, stageAttempt) to executor metric peaks per executor/driver for the stage private val liveStageExecutorMetrics = Map.empty[(Int, Int), Map[String, ExecutorMetrics]] /** @@ -176,13 +176,11 @@ private[spark] class EventLoggingListener( override def onTaskEnd(event: SparkListenerTaskEnd): Unit = { logEvent(event) if (shouldLogStageExecutorMetrics) { - val k1 = (event.stageId, event.stageAttemptId) - liveStageExecutorMetrics.foreach { case (k2, peakExecutorMetrics) => - if (k1 == k2) { - val peakMetrics = peakExecutorMetrics.getOrElseUpdate( - event.taskInfo.executorId, new ExecutorMetrics()) - peakMetrics.compareAndUpdatePeakValues(event.taskExecutorMetrics) - } + val stageKey = (event.stageId, event.stageAttemptId) + liveStageExecutorMetrics.get(stageKey).map { metricsPerExecutor => + val metrics = metricsPerExecutor.getOrElseUpdate( + event.taskInfo.executorId, new ExecutorMetrics()) + metrics.compareAndUpdatePeakValues(event.taskExecutorMetrics) } } } @@ -280,16 +278,16 @@ private[spark] class EventLoggingListener( override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { if (shouldLogStageExecutorMetrics) { - event.executorUpdates.foreach { case (k1, peakUpdates) => - liveStageExecutorMetrics.foreach { case (k2, peakExecutorMetrics) => - // If the update came from the driver, the key k1 will be the dummy key (-1, -1), + event.executorUpdates.foreach { case (stageKey1, peaks) => + liveStageExecutorMetrics.foreach { case (stageKey2, metricsPerExecutor) => + // If the update came from the driver, stageKey1 will be the dummy key (-1, -1), // so record those peaks for all active stages. // Otherwise, record the peaks for the matching stage. - val k0 = (-1, -1) - if (k1 == k0 || k1 == k2) { - val peakMetrics = peakExecutorMetrics.getOrElseUpdate( + val driverStageKey = (-1, -1) + if (stageKey1 == driverStageKey || stageKey1 == stageKey2) { + val metrics = metricsPerExecutor.getOrElseUpdate( event.execId, new ExecutorMetrics()) - peakMetrics.compareAndUpdatePeakValues(peakUpdates) + metrics.compareAndUpdatePeakValues(peaks) } } } From ea2ff0d537e0e6f4935a36a4181c5b4755aedfad Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Mon, 18 Mar 2019 15:26:01 -0700 Subject: [PATCH 07/22] [SPARK-26329][CORE] On task failure, send executor metrics in the TaskFailedReason. Do this in the cases where we currently send accumulator updates. --- .../scala/org/apache/spark/TaskEndReason.scala | 11 +++++++++-- .../org/apache/spark/executor/Executor.scala | 17 ++++++++++++----- .../apache/spark/scheduler/TaskSetManager.scala | 4 +++- .../apache/spark/executor/ExecutorSuite.scala | 3 ++- 4 files changed, 26 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 33901bc8380e..19f71a1dec29 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -128,7 +128,8 @@ case class ExceptionFailure( fullStackTrace: String, private val exceptionWrapper: Option[ThrowableSerializationWrapper], accumUpdates: Seq[AccumulableInfo] = Seq.empty, - private[spark] var accums: Seq[AccumulatorV2[_, _]] = Nil) + private[spark] var accums: Seq[AccumulatorV2[_, _]] = Nil, + private[spark] var metricPeaks: Seq[Long] = Seq.empty) extends TaskFailedReason { /** @@ -153,6 +154,11 @@ case class ExceptionFailure( this } + private[spark] def withMetricPeaks(metricPeaks: Seq[Long]): ExceptionFailure = { + this.metricPeaks = metricPeaks + this + } + def exception: Option[Throwable] = exceptionWrapper.flatMap(w => Option(w.exception)) override def toErrorString: String = @@ -215,7 +221,8 @@ case object TaskResultLost extends TaskFailedReason { case class TaskKilled( reason: String, accumUpdates: Seq[AccumulableInfo] = Seq.empty, - private[spark] val accums: Seq[AccumulatorV2[_, _]] = Nil) + private[spark] val accums: Seq[AccumulatorV2[_, _]] = Nil, + metricPeaks: Seq[Long] = Seq.empty) extends TaskFailedReason { override def toErrorString: String = s"TaskKilled ($reason)" 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 ff67dbe837cd..28c1d5afc34e 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -27,7 +27,7 @@ import java.util.concurrent._ import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, Map, WrappedArray} import scala.concurrent.duration._ import scala.util.control.NonFatal @@ -590,7 +590,8 @@ private[spark] class Executor( logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}") val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) - val serializedTK = ser.serialize(TaskKilled(t.reason, accUpdates, accums)) + val metricPeaks = WrappedArray.make(metricsPoller.getTaskMetricPeaks(taskId)) + val serializedTK = ser.serialize(TaskKilled(t.reason, accUpdates, accums, metricPeaks)) execBackend.statusUpdate(taskId, TaskState.KILLED, serializedTK) case _: InterruptedException | NonFatal(_) if @@ -599,7 +600,8 @@ private[spark] class Executor( logInfo(s"Executor interrupted and killed $taskName (TID $taskId), reason: $killReason") val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) - val serializedTK = ser.serialize(TaskKilled(killReason, accUpdates, accums)) + val metricPeaks = WrappedArray.make(metricsPoller.getTaskMetricPeaks(taskId)) + val serializedTK = ser.serialize(TaskKilled(killReason, accUpdates, accums, metricPeaks)) execBackend.statusUpdate(taskId, TaskState.KILLED, serializedTK) case t: Throwable if hasFetchFailure && !Utils.isFatalError(t) => @@ -634,14 +636,19 @@ private[spark] class Executor( // instead of an app issue). if (!ShutdownHookManager.inShutdown()) { val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) + val metricPeaks = WrappedArray.make(metricsPoller.getTaskMetricPeaks(taskId)) val serializedTaskEndReason = { try { - ser.serialize(new ExceptionFailure(t, accUpdates).withAccums(accums)) + val ef = new ExceptionFailure(t, accUpdates).withAccums(accums) + .withMetricPeaks(metricPeaks) + ser.serialize(ef) } catch { case _: NotSerializableException => // t is not serializable so just send the stacktrace - ser.serialize(new ExceptionFailure(t, accUpdates, false).withAccums(accums)) + val ef = new ExceptionFailure(t, accUpdates, false).withAccums(accums) + .withMetricPeaks(metricPeaks) + ser.serialize(ef) } } setTaskFinishedAndClearInterruptStatus() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c8a7c729d043..f4b96d03674c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -859,7 +859,7 @@ private[spark] class TaskSetManager( val index = info.index copiesRunning(index) -= 1 var accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty - val metricPeaks: Array[Long] = Array.empty + var metricPeaks: Array[Long] = Array.empty val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}," + s" executor ${info.executorId}): ${reason.toErrorString}" val failureException: Option[Throwable] = reason match { @@ -881,6 +881,7 @@ private[spark] class TaskSetManager( case ef: ExceptionFailure => // ExceptionFailure's might have accumulator updates accumUpdates = ef.accums + metricPeaks = ef.metricPeaks.toArray if (ef.className == classOf[NotSerializableException].getName) { // If the task result wasn't serializable, there's no point in trying to re-execute it. logError("Task %s in stage %s (TID %d) had a not serializable result: %s; not retrying" @@ -918,6 +919,7 @@ private[spark] class TaskSetManager( case tk: TaskKilled => // TaskKilled might have accumulator updates accumUpdates = tk.accums + metricPeaks = tk.metricPeaks.toArray logWarning(failureReason) None diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 8edf95763db4..0311150e1e68 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -126,7 +126,8 @@ class ExecutorSuite extends SparkFunSuite } // `testFailedReason` should be `TaskKilled`; `taskState` should be `KILLED` - assert(executorSuiteHelper.testFailedReason === TaskKilled("test")) + assert(executorSuiteHelper.testFailedReason.isInstanceOf[TaskKilled]) + assert(executorSuiteHelper.testFailedReason.toErrorString === "TaskKilled (test)") assert(executorSuiteHelper.taskState === TaskState.KILLED) } finally { From 0cbfc0425116d9da8bf5c1802b8d02a309e7ea67 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 20 Mar 2019 17:49:01 -0700 Subject: [PATCH 08/22] [SPARK-26329][CORE] Unit tests for sending executor metrics in TaskResult and TaskFailedReason. --- .../apache/spark/executor/ExecutorSuite.scala | 155 +++++++++++++----- 1 file changed, 115 insertions(+), 40 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 0311150e1e68..366a8e63f829 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -25,8 +25,7 @@ import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean import scala.collection.immutable -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.Map +import scala.collection.mutable.{ArrayBuffer, Map} import scala.concurrent.duration._ import org.mockito.ArgumentCaptor @@ -47,8 +46,8 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rdd.RDD import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcTimeout} -import org.apache.spark.scheduler.{FakeTask, ResultTask, Task, TaskDescription} -import org.apache.spark.serializer.{JavaSerializer, SerializerManager} +import org.apache.spark.scheduler.{DirectTaskResult, FakeTask, ResultTask, Task, TaskDescription} +import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockManager, BlockManagerId} import org.apache.spark.util.{LongAccumulator, UninterruptibleThread} @@ -148,21 +147,7 @@ class ExecutorSuite extends SparkFunSuite // fetch failure, not a generic exception from user code. val inputRDD = new FetchFailureThrowingRDD(sc) val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = false, interrupt = false) - val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array()) - val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array() - val task = new ResultTask( - stageId = 1, - stageAttemptId = 0, - taskBinary = taskBinary, - partition = secondRDD.partitions(0), - locs = Seq(), - outputId = 0, - localProperties = new Properties(), - serializedTaskMetrics = serializedTaskMetrics - ) - - val serTask = serializer.serialize(task) - val taskDescription = createFakeTaskDescription(serTask) + val taskDescription = createResultTaskDescription(serializer, resultFunc, secondRDD, 1) val failReason = runTaskAndGetFailReason(taskDescription) assert(failReason.isInstanceOf[FetchFailed]) @@ -197,13 +182,15 @@ class ExecutorSuite extends SparkFunSuite } /** - * Helper for testing some cases where a FetchFailure should *not* get sent back, because its - * superceded by another error, either an OOM or intentionally killing a task. + * Helper for testing some cases where a FetchFailure should *not* get sent back, because it's + * superseded by another error, either an OOM or intentionally killing a task. * @param oom if true, throw an OOM after the FetchFailure; else, interrupt the task after the - * FetchFailure + * FetchFailure + * @param poll if true, poll executor metrics after launching task */ private def testFetchFailureHandling( - oom: Boolean): (TaskFailedReason, UncaughtExceptionHandler) = { + oom: Boolean, + poll: Boolean = false): (TaskFailedReason, UncaughtExceptionHandler) = { // when there is a fatal error like an OOM, we don't do normal fetch failure handling, since it // may be a false positive. And we should call the uncaught exception handler. // SPARK-23816 also handle interrupts the same way, as killing an obsolete speculative task @@ -223,23 +210,9 @@ class ExecutorSuite extends SparkFunSuite ExecutorSuiteHelper.latches = new ExecutorSuiteHelper() } val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = oom, interrupt = !oom) - val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array()) - val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array() - val task = new ResultTask( - stageId = 1, - stageAttemptId = 0, - taskBinary = taskBinary, - partition = secondRDD.partitions(0), - locs = Seq(), - outputId = 0, - localProperties = new Properties(), - serializedTaskMetrics = serializedTaskMetrics - ) - - val serTask = serializer.serialize(task) - val taskDescription = createFakeTaskDescription(serTask) + val taskDescription = createResultTaskDescription(serializer, resultFunc, secondRDD, 1) - runTaskGetFailReasonAndExceptionHandler(taskDescription, killTask = !oom) + runTaskGetFailReasonAndExceptionHandler(taskDescription, killTask = !oom, poll) } test("Gracefully handle error in task deserialization") { @@ -343,6 +316,75 @@ class ExecutorSuite extends SparkFunSuite } } + test("Send task executor metrics in DirectTaskResult") { + // Run a successful, trivial result task + val conf = new SparkConf().setMaster("local").setAppName("executor suite test") + sc = new SparkContext(conf) + val serializer = SparkEnv.get.closureSerializer.newInstance() + val resultFunc = (context: TaskContext, itr: Iterator[Int]) => itr.size + val rdd = new RDD[Int](sc, Nil) { + override def compute(split: Partition, context: TaskContext): Iterator[Int] = { + val l = List(1) + l.iterator + } + override protected def getPartitions: Array[Partition] = { + Array(new SimplePartition) + } + } + val taskDescription = createResultTaskDescription(serializer, resultFunc, rdd, 0) + + val mockBackend = mock[ExecutorBackend] + var executor: Executor = null + try { + executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true) + executor.launchTask(mockBackend, taskDescription) + + // Ensure that the executor's metricsPoller is polled at least once so that values are + // recorded for the task metrics + val executorClass = classOf[Executor] + val metricsPollerField = + executorClass.getDeclaredField("org$apache$spark$executor$Executor$$metricsPoller") + metricsPollerField.setAccessible(true) + val metricsPoller = metricsPollerField.get(executor).asInstanceOf[ExecutorMetricsPoller] + metricsPoller.poll() + eventually(timeout(1.seconds), interval(10.milliseconds)) { + assert(executor.numRunningTasks === 0) + } + } finally { + if (executor != null) { + executor.stop() + } + } + + // Verify that peak values for task metrics get sent in the TaskResult + val orderedMock = inOrder(mockBackend) + val statusCaptor = ArgumentCaptor.forClass(classOf[ByteBuffer]) + orderedMock.verify(mockBackend) + .statusUpdate(meq(0L), meq(TaskState.RUNNING), statusCaptor.capture()) + orderedMock.verify(mockBackend) + .statusUpdate(meq(0L), meq(TaskState.FINISHED), statusCaptor.capture()) + val resultData = statusCaptor.getAllValues.get(1) + val result = serializer.deserialize[DirectTaskResult[Int]](resultData) + val taskMetrics = new ExecutorMetrics(result.metricPeaks) + assert(taskMetrics.getMetricValue("JVMHeapMemory") > 0) + } + + test("Send task executor metrics in TaskKilled") { + val (taskFailedReason, _) = testFetchFailureHandling(false, true) + assert(taskFailedReason.isInstanceOf[TaskKilled]) + val metrics = taskFailedReason.asInstanceOf[TaskKilled].metricPeaks.toArray + val taskMetrics = new ExecutorMetrics(metrics) + assert(taskMetrics.getMetricValue("JVMHeapMemory") > 0) + } + + test("Send task executor metrics in ExceptionFailure") { + val (taskFailedReason, _) = testFetchFailureHandling(true, true) + assert(taskFailedReason.isInstanceOf[ExceptionFailure]) + val metrics = taskFailedReason.asInstanceOf[ExceptionFailure].metricPeaks.toArray + val taskMetrics = new ExecutorMetrics(metrics) + assert(taskMetrics.getMetricValue("JVMHeapMemory") > 0) + } + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer): SparkEnv = { val mockEnv = mock[SparkEnv] val mockRpcEnv = mock[RpcEnv] @@ -361,6 +403,27 @@ class ExecutorSuite extends SparkFunSuite mockEnv } + private def createResultTaskDescription( + serializer: SerializerInstance, + resultFunc: (TaskContext, Iterator[Int]) => Int, + rdd: RDD[Int], + stageId: Int): TaskDescription = { + val taskBinary = sc.broadcast(serializer.serialize((rdd, resultFunc)).array()) + val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array() + val task = new ResultTask( + stageId = stageId, + stageAttemptId = 0, + taskBinary = taskBinary, + partition = rdd.partitions(0), + locs = Seq(), + outputId = 0, + localProperties = new Properties(), + serializedTaskMetrics = serializedTaskMetrics + ) + val serTask = serializer.serialize(task) + createFakeTaskDescription(serTask) + } + private def createFakeTaskDescription(serializedTask: ByteBuffer): TaskDescription = { new TaskDescription( taskId = 0, @@ -382,7 +445,8 @@ class ExecutorSuite extends SparkFunSuite private def runTaskGetFailReasonAndExceptionHandler( taskDescription: TaskDescription, - killTask: Boolean): (TaskFailedReason, UncaughtExceptionHandler) = { + killTask: Boolean, + poll: Boolean = false): (TaskFailedReason, UncaughtExceptionHandler) = { val mockBackend = mock[ExecutorBackend] val mockUncaughtExceptionHandler = mock[UncaughtExceptionHandler] var executor: Executor = null @@ -392,6 +456,17 @@ class ExecutorSuite extends SparkFunSuite uncaughtExceptionHandler = mockUncaughtExceptionHandler) // the task will be launched in a dedicated worker thread executor.launchTask(mockBackend, taskDescription) + if (poll) { + // Ensure that the executor's metricsPoller is polled at least once so that values are + // recorded for the task metrics. + // When the task fails, peak values for these metrics get sent in the TaskFailedReason. + val executorClass = classOf[Executor] + val metricsPollerField = + executorClass.getDeclaredField("org$apache$spark$executor$Executor$$metricsPoller") + metricsPollerField.setAccessible(true) + val metricsPoller = metricsPollerField.get(executor).asInstanceOf[ExecutorMetricsPoller] + metricsPoller.poll() + } if (killTask) { val killingThread = new Thread("kill-task") { override def run(): Unit = { From 8cb30a876151a7a6c155c2e3ba2fe09f6dfb3984 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Fri, 22 Mar 2019 12:36:54 -0700 Subject: [PATCH 09/22] [SPARK-26329][CORE] Add driver updates to test for executor metrics aggregation. Add an internal check within the test showing how the expected aggregated metrics are calculated. This documents how the expected values are arrived at. --- .../scheduler/EventLoggingListenerSuite.scala | 185 ++++++++++++------ 1 file changed, 124 insertions(+), 61 deletions(-) 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 f7de8745ca74..1d73d0d2fd27 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import java.io.{File, FileOutputStream, InputStream, IOException} +import java.util.Arrays import scala.collection.immutable.Map import scala.collection.mutable @@ -288,6 +289,79 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf) val listenerBus = new LiveListenerBus(conf) + // Executor metrics + // driver + val md_1 = Array(4000L, 50L, 0L, 0L, 40L, 0L, 40L, 0L, 70L, 0L, 7500L, 3500L, + 0L, 0L, 0L, 0L, 10L, 90L, 2L, 20L) + val md_2 = Array(4500L, 50L, 0L, 0L, 40L, 0L, 40L, 0L, 70L, 0L, 8000L, 3500L, + 0L, 0L, 0L, 0L, 10L, 90L, 3L, 20L) + val md_3 = Array(4200L, 50L, 0L, 0L, 40L, 0L, 40L, 0L, 70L, 0L, 7800L, 3500L, + 0L, 0L, 0L, 0L, 15L, 100L, 5L, 20L) + + // executors 1 and 2 + val m1_1 = Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L, 10L, 90L, 2L, 20L) + val m2_1 = Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L) + val m1_2 = Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L) + val m2_2 = Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L) + val m1_3 = Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L, 10L, 90L, 2L, 20L) + val m2_3 = Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L) + val m1_4 = Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, + 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L) + val m2_4 = Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, + 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L) + val m1_5 = Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L) + val m2_5 = Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, + 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L) + val m1_6 = Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, + 2000L, 1500L, 1000L, 500L, 10L, 90L, 2L, 20L) + val m2_6 = Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, + 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L, 10L, 90L, 2L, 20L) + val m1_7 = Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, + 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L, 10L, 90L, 2L, 20L) + val m2_7 = Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, + 4000L, 6000L, 3000L, 5000L, 2000L, 10L, 90L, 2L, 20L) + + def max(a: Array[Long], b: Array[Long]): Array[Long] = + (a, b).zipped.map(Math.max) + + // calculated metric peaks per stage per executor + // metrics sent during stage 0 for each executor + val cp0_1 = Seq(m1_1, m1_2, m1_3, m1_4).reduceLeft(max) + val cp0_2 = Seq(m2_1, m2_2, m2_3, m2_4).reduceLeft(max) + val cp0_d = Seq(md_1, md_2).reduceLeft(max) + // metrics sent during stage 1 for each executor + val cp1_1 = Seq(m1_4, m1_5, m1_6, m1_7).reduceLeft(max) + val cp1_2 = Seq(m2_4, m2_5, m2_6, m2_7).reduceLeft(max) + val cp1_d = Seq(md_2, md_3).reduceLeft(max) + + // expected metric peaks per stage per executor + val p0_1 = Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L) + val p0_2 = Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L) + val p0_d = Array(4500L, 50L, 0L, 0L, 40L, 0L, 40L, 0L, + 70L, 0L, 8000L, 3500L, 0L, 0L, 0L, 0L, 10L, 90L, 3L, 20L) + val p1_1 = Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, + 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L) + val p1_2 = Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, + 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L) + val p1_d = Array(4500L, 50L, 0L, 0L, 40L, 0L, 40L, 0L, + 70L, 0L, 8000L, 3500L, 0L, 0L, 0L, 0L, 15L, 100L, 5L, 20L) + + assert(java.util.Arrays.equals(p0_1, cp0_1)) + assert(java.util.Arrays.equals(p0_2, cp0_2)) + assert(java.util.Arrays.equals(p0_d, cp0_d)) + assert(java.util.Arrays.equals(p1_1, cp1_1)) + assert(java.util.Arrays.equals(p1_2, cp1_2)) + assert(java.util.Arrays.equals(p1_d, cp1_d)) + // Events to post. val events = Array( SparkListenerApplicationStart("executionMetrics", None, @@ -297,68 +371,47 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit createStageSubmittedEvent(0), // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor - createExecutorMetricsUpdateEvent(List(0), 1, - new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, - 6500L, 2500L, 5500L, 1500L, 10L, 90L, 2L, 20L))), - createExecutorMetricsUpdateEvent(List(0), 2, - new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, - 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))), - // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 - createExecutorMetricsUpdateEvent(List(0), 1, - new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, - 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))), + // also, receive 1 metric update from the driver + createExecutorMetricsUpdateEvent(List(0), "1", new ExecutorMetrics(m1_1)), + createExecutorMetricsUpdateEvent(List(0), "2", new ExecutorMetrics(m2_1)), + // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 + createExecutorMetricsUpdateEvent(List(0), "1", new ExecutorMetrics(m1_2)), // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 - createExecutorMetricsUpdateEvent(List(0), 2, - new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, - 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(0), "2", new ExecutorMetrics(m2_2)), + // driver + createExecutorMetricsUpdateEvent(List(-1), "driver", new ExecutorMetrics(md_1)), // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 - createExecutorMetricsUpdateEvent(List(0), 1, - new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, - 7000L, 2500L, 6000L, 1500L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(0), "1", new ExecutorMetrics(m1_3)), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 - createExecutorMetricsUpdateEvent(List(0), 2, - new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, - 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(0), "2", new ExecutorMetrics(m2_3)), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks - createExecutorMetricsUpdateEvent(List(0, 1), 1, - new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, - 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(0, 1), "1", new ExecutorMetrics(m1_4)), // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 3, 6, 7, 9; // initialize stage 1 peaks - createExecutorMetricsUpdateEvent(List(0, 1), 2, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, - 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(0, 1), "2", new ExecutorMetrics(m2_4)), + // driver + createExecutorMetricsUpdateEvent(List(-1), "driver", new ExecutorMetrics(md_2)), // 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(List(1), 1, - new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, - 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(1), "1", new ExecutorMetrics(m1_5)), // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 - createExecutorMetricsUpdateEvent(List(1), 2, - new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, - 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(1), "2", new ExecutorMetrics(m2_5)), // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 - createExecutorMetricsUpdateEvent(List(1), 1, - new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, - 2000L, 1500L, 1000L, 500L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(1), "1", new ExecutorMetrics(m1_6)), // exec 2: new stage 1 peak for metrics at index: 7 - createExecutorMetricsUpdateEvent(List(1), 2, - new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, - 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(1), "2", new ExecutorMetrics(m2_6)), + // driver + createExecutorMetricsUpdateEvent(List(-1), "driver", new ExecutorMetrics(md_3)), // exec 1: no new stage 1 peaks - createExecutorMetricsUpdateEvent(List(1), 1, - new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, - 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(1), "1", new ExecutorMetrics(m1_7)), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 - createExecutorMetricsUpdateEvent(List(1), 2, - new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, - 4000L, 6000L, 3000L, 5000L, 2000L, 10L, 90L, 2L, 20L))), + createExecutorMetricsUpdateEvent(List(1), "2", new ExecutorMetrics(m2_7)), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -374,21 +427,17 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit 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, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L)))), + new SparkListenerStageExecutorMetrics("1", 0, 0, new ExecutorMetrics(p0_1))), ((0, "2"), - new SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, - 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L)))), + new SparkListenerStageExecutorMetrics("2", 0, 0, new ExecutorMetrics(p0_2))), + ((0, "driver"), + new SparkListenerStageExecutorMetrics("driver", 0, 0, new ExecutorMetrics(p0_d))), ((1, "1"), - new SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, - 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L)))), + new SparkListenerStageExecutorMetrics("1", 1, 0, new ExecutorMetrics(p1_1))), ((1, "2"), - new SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, - 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L))))) + new SparkListenerStageExecutorMetrics("2", 1, 0, new ExecutorMetrics(p1_2))), + ((1, "driver"), + new SparkListenerStageExecutorMetrics("driver", 1, 0, new ExecutorMetrics(p1_d)))) // 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. @@ -396,7 +445,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit try { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) - assert(lines.size === 14) + assert(lines.size === 16) assert(lines(0).contains("SparkListenerLogStart")) assert(lines(1).contains("SparkListenerApplicationStart")) assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) @@ -406,13 +455,13 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit case metricsUpdate: SparkListenerExecutorMetricsUpdate => case stageCompleted: SparkListenerStageCompleted => val execIds = Set[String]() - (1 to 2).foreach { _ => + (1 to 3).foreach { _ => val execId = checkStageExecutorMetrics(lines(logIdx), stageCompleted.stageInfo.stageId, expectedMetricsEvents) execIds += execId logIdx += 1 } - assert(execIds.size == 2) // check that each executor was logged + assert(execIds.size == 3) // check that each executor/driver was logged checkEvent(lines(logIdx), event) logIdx += 1 case _ => @@ -444,16 +493,30 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit SparkListenerExecutorRemoved(0L, executorId.toString, "test") } + /** + * Helper to create a SparkListenerExecutorMetricsUpdate event. + * For the driver (executorId == "driver"), the executorUpdates contain a single entry with + * the key (-1, -1). There should be a single stageId passed in stageIds, namely -1. + * For the executors, for each stage, we assume there is a single stage attempt (attempt 0); + * the executorUpdates contain an entry for each stageId passed in stageIds, with the key + * (stageId, 0). + * The same executorMetrics are associated to each key in the executorUpdates. + */ private def createExecutorMetricsUpdateEvent( stageIds: Seq[Int], - executorId: Int, + executorId: String, 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 = stageIds.map(id => (id, 0) -> executorMetrics).toMap - SparkListenerExecutorMetricsUpdate(executorId.toString, accum, executorUpdates) + val executorUpdates = + if (executorId == "driver") { + stageIds.map(id => (id, -1) -> executorMetrics).toMap + } else { + stageIds.map(id => (id, 0) -> executorMetrics).toMap + } + SparkListenerExecutorMetricsUpdate(executorId, accum, executorUpdates) } /** Check that the Spark history log line matches the expected event. */ From 077abb0874fd8a5392e9ff5b56e93abc31788112 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Fri, 22 Mar 2019 19:08:28 -0700 Subject: [PATCH 10/22] [SPARK-26329][CORE] Add SparkListenerTaskEnd events to test for executor metrics aggregation. --- .../scheduler/EventLoggingListenerSuite.scala | 72 +++++++++++++++---- 1 file changed, 57 insertions(+), 15 deletions(-) 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 1d73d0d2fd27..df1e79fec542 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -280,7 +280,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit /** * Test stage executor metrics logging functionality. This checks that peak - * values from SparkListenerExecutorMetricsUpdate events during a stage are + * values from SparkListenerExecutorMetricsUpdate events during a stage and + * from SparkListenerTaskEnd events for tasks belonging to the stage are * logged in a StageExecutorMetrics event for each executor at stage completion. */ private def testStageExecutorMetricsEventLogging() { @@ -328,39 +329,53 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val m2_7 = Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, 4000L, 6000L, 3000L, 5000L, 2000L, 10L, 90L, 2L, 20L) + // tasks + val t1 = Array(4500L, 60L, 50L, 0L, 50L, 10L, 100L, 10L, 70L, 20L, + 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L) + val t2 = Array(3500L, 50L, 20L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, + 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L) + val t3 = Array(5000L, 60L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L, + 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L) + val t4 = Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L, + 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L) + val t5 = Array(7000L, 100L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L, + 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L) + val t6 = Array(7200L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L, + 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L) + def max(a: Array[Long], b: Array[Long]): Array[Long] = (a, b).zipped.map(Math.max) // calculated metric peaks per stage per executor // metrics sent during stage 0 for each executor - val cp0_1 = Seq(m1_1, m1_2, m1_3, m1_4).reduceLeft(max) - val cp0_2 = Seq(m2_1, m2_2, m2_3, m2_4).reduceLeft(max) + val cp0_1 = Seq(m1_1, m1_2, m1_3, t1, m1_4, t3).reduceLeft(max) + val cp0_2 = Seq(m2_1, m2_2, m2_3, t2, m2_4, t4).reduceLeft(max) val cp0_d = Seq(md_1, md_2).reduceLeft(max) // metrics sent during stage 1 for each executor - val cp1_1 = Seq(m1_4, m1_5, m1_6, m1_7).reduceLeft(max) - val cp1_2 = Seq(m2_4, m2_5, m2_6, m2_7).reduceLeft(max) + val cp1_1 = Seq(m1_4, m1_5, m1_6, m1_7, t5).reduceLeft(max) + val cp1_2 = Seq(m2_4, m2_5, m2_6, m2_7, t6).reduceLeft(max) val cp1_d = Seq(md_2, md_3).reduceLeft(max) // expected metric peaks per stage per executor - val p0_1 = Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + val p0_1 = Array(5000L, 60L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L) val p0_2 = Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L) val p0_d = Array(4500L, 50L, 0L, 0L, 40L, 0L, 40L, 0L, 70L, 0L, 8000L, 3500L, 0L, 0L, 0L, 0L, 10L, 90L, 3L, 20L) - val p1_1 = Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, + val p1_1 = Array(7000L, 100L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L) - val p1_2 = Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, + val p1_2 = Array(7200L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L) val p1_d = Array(4500L, 50L, 0L, 0L, 40L, 0L, 40L, 0L, 70L, 0L, 8000L, 3500L, 0L, 0L, 0L, 0L, 15L, 100L, 5L, 20L) - assert(java.util.Arrays.equals(p0_1, cp0_1)) - assert(java.util.Arrays.equals(p0_2, cp0_2)) - assert(java.util.Arrays.equals(p0_d, cp0_d)) - assert(java.util.Arrays.equals(p1_1, cp1_1)) - assert(java.util.Arrays.equals(p1_2, cp1_2)) - assert(java.util.Arrays.equals(p1_d, cp1_d)) + assert(Arrays.equals(p0_1, cp0_1)) + assert(Arrays.equals(p0_2, cp0_2)) + assert(Arrays.equals(p0_d, cp0_d)) + assert(Arrays.equals(p1_1, cp1_1)) + assert(Arrays.equals(p1_2, cp1_2)) + assert(Arrays.equals(p1_d, cp1_d)) // Events to post. val events = Array( @@ -384,6 +399,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit createExecutorMetricsUpdateEvent(List(0), "1", new ExecutorMetrics(m1_3)), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(List(0), "2", new ExecutorMetrics(m2_3)), + // stage 0: task 1 (on exec 1) and task 2 (on exec 2) end + createTaskEndEvent(1L, 0, "1", 0, "ShuffleMapTask", new ExecutorMetrics(t1)), + createTaskEndEvent(2L, 0, "2", 0, "ShuffleMapTask", new ExecutorMetrics(t2)), // 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), @@ -394,6 +412,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit createExecutorMetricsUpdateEvent(List(0, 1), "2", new ExecutorMetrics(m2_4)), // driver createExecutorMetricsUpdateEvent(List(-1), "driver", new ExecutorMetrics(md_2)), + // stage 0: task 3 (on exec 1) and task 4 (on exec 2) end + createTaskEndEvent(3L, 1, "1", 0, "ShuffleMapTask", new ExecutorMetrics(t3)), + createTaskEndEvent(4L, 1, "2", 0, "ShuffleMapTask", new ExecutorMetrics(t4)), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), @@ -409,9 +430,13 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit createExecutorMetricsUpdateEvent(List(-1), "driver", new ExecutorMetrics(md_3)), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(List(1), "1", new ExecutorMetrics(m1_7)), + // stage 1: task 5 (on exec 1) end; new stage 1 peaks at index: 1 + createTaskEndEvent(5L, 2, "1", 1, "ResultTask", new ExecutorMetrics(t5)), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(List(1), "2", new ExecutorMetrics(m2_7)), + // stage 1: task 6 (on exec 2) end; new stage 2 peaks at index: 0 + createTaskEndEvent(6L, 2, "2", 1, "ResultTask", new ExecutorMetrics(t6)), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -445,7 +470,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit try { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) - assert(lines.size === 16) + assert(lines.size === 22) assert(lines(0).contains("SparkListenerLogStart")) assert(lines(1).contains("SparkListenerApplicationStart")) assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) @@ -519,6 +544,19 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit SparkListenerExecutorMetricsUpdate(executorId, accum, executorUpdates) } + private def createTaskEndEvent( + taskId: Long, + taskIndex: Int, + executorId: String, + stageId: Int, + taskType: String, + executorMetrics: ExecutorMetrics): SparkListenerTaskEnd = { + val taskInfo = new TaskInfo(taskId, taskIndex, 0, 1553291556000L, executorId, "executor", + TaskLocality.NODE_LOCAL, false) + val taskMetrics = TaskMetrics.empty + SparkListenerTaskEnd(stageId, 0, taskType, Success, taskInfo, executorMetrics, taskMetrics) + } + /** 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)) @@ -531,6 +569,10 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit case (expected: SparkListenerStageCompleted, actual: SparkListenerStageCompleted) => // accumulables can be different, so only check the stage Id assert(expected.stageInfo.stageId == actual.stageInfo.stageId) + case (expected: SparkListenerTaskEnd, actual: SparkListenerTaskEnd) => + // taskInfo, taskExecutorMetrics, and taskMetrics will be different object references, + // so only check stageId + assert(expected.stageId == actual.stageId) case (expected: SparkListenerEvent, actual: SparkListenerEvent) => assert(expected === actual) } From 7a3c90dbbb0431bcb669fa4075d5e194099945cf Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Tue, 26 Mar 2019 18:14:15 -0700 Subject: [PATCH 11/22] [SPARK-26329][CORE] Address feedback from irashid. In ExecutorMetricsPoller, combine onTaskLaunch and onTaskStart, and combine onTaskCompletion and onTaskCleanup. Simplify getExecutorUpdates, as there should never be a stage in stageTCMP with task count == 0. Some code format cleanup and minor improvements. --- .../org/apache/spark/executor/Executor.scala | 18 ++++--- .../spark/executor/ExecutorMetrics.scala | 2 +- .../executor/ExecutorMetricsPoller.scala | 50 +++++++------------ .../apache/spark/scheduler/TaskResult.scala | 7 ++- .../apache/spark/executor/ExecutorSuite.scala | 14 +----- .../scheduler/EventLoggingListenerSuite.scala | 14 +++--- 6 files changed, 44 insertions(+), 61 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 28c1d5afc34e..7e476da9a8b5 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -195,8 +195,8 @@ private[spark] class Executor( private val pollOnHeartbeat = if (METRICS_POLLING_INTERVAL_MS > 0) false else true - // Poller for the memory metrics. - private val metricsPoller = new ExecutorMetricsPoller( + // Poller for the memory metrics. Visible for testing. + private[executor] val metricsPoller = new ExecutorMetricsPoller( env.memoryManager, METRICS_POLLING_INTERVAL_MS) @@ -225,7 +225,6 @@ private[spark] class Executor( def launchTask(context: ExecutorBackend, taskDescription: TaskDescription): Unit = { val tr = new TaskRunner(context, taskDescription) runningTasks.put(taskDescription.taskId, tr) - metricsPoller.onTaskLaunch(taskDescription.taskId) threadPool.execute(tr) } @@ -400,6 +399,7 @@ private[spark] class Executor( var taskStartTimeNs: Long = 0 var taskStartCpu: Long = 0 startGCTime = computeTotalGcTime() + var taskStarted: Boolean = false try { // Must be set before updateDependencies() is called, in case fetching dependencies @@ -434,7 +434,8 @@ private[spark] class Executor( val stageId = task.stageId val stageAttemptId = task.stageAttemptId - metricsPoller.onTaskStart(stageId, stageAttemptId) + metricsPoller.onTaskStart(taskId, stageId, stageAttemptId) + taskStarted = true // Run the actual task and measure its runtime. taskStartTimeNs = System.nanoTime() @@ -581,7 +582,6 @@ private[spark] class Executor( } } - metricsPoller.onTaskCompletion(stageId, stageAttemptId) executorSource.SUCCEEDED_TASKS.inc(1L) setTaskFinishedAndClearInterruptStatus() execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) @@ -664,7 +664,13 @@ private[spark] class Executor( } } finally { runningTasks.remove(taskId) - metricsPoller.onTaskCleanup(taskId) + if (taskStarted) { + // This means the task was successfully deserialized, the stageId and stageAttemptId + // are known, and metricsPoller.onTaskStart was called. + val stageId = task.stageId + val stageAttemptId = task.stageAttemptId + metricsPoller.onTaskCompletion(taskId, stageId, stageAttemptId) + } } } 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 78adfe632635..d9aa3ef60fc9 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -57,7 +57,7 @@ class ExecutorMetrics private[spark] extends Serializable { } /** - * Constructor: create the ExecutorMetrics with using a given map. + * Constructor: create the ExecutorMetrics using a given map. * * @param executorMetrics map of executor metric name to value */ diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala index 07ff7735e7ec..a6a98c1e8654 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -32,6 +32,12 @@ import org.apache.spark.util.{ThreadUtils, Utils} * :: DeveloperApi :: * A class that polls executor metrics, and tracks their peaks per task and per stage. * Each executor keeps an instance of this class. + * The poll method polls the executor metrics, and is either run in its own thread or + * called by the executor's heartbeater thread, depending on configuration. + * The class keeps two ConcurrentHashMaps that are accessed (via its methods) by the + * executor's task runner threads concurrently with the polling thread. One thread may + * update one of these maps while another reads it, so the reading thread may not get + * the latest metrics, but this is ok. * * @param memoryManager the memory manager used by the executor. * @param pollingInterval the polling interval in milliseconds. @@ -54,7 +60,7 @@ private[spark] class ExecutorMetricsPoller( private val poller = if (pollingInterval > 0) { - ThreadUtils.newDaemonSingleThreadScheduledExecutor("executor-poller") + ThreadUtils.newDaemonSingleThreadScheduledExecutor("executor-metrics-poller") } else { null } @@ -98,22 +104,17 @@ private[spark] class ExecutorMetricsPoller( } } - /** - * Called by Executor#launchTask. - * - * @param taskId the id of the task being launched. - */ - def onTaskLaunch(taskId: Long): Unit = { - taskMetricPeaks.put(taskId, new AtomicLongArray(ExecutorMetricType.numMetrics)) - } - /** * Called by TaskRunner#run. * + * @param taskId the id of the task being launched. * @param stageId the id of the stage the task belongs to. * @param stageAttemptId the attempt number of the stage the task belongs to. */ - def onTaskStart(stageId: Int, stageAttemptId: Int): Unit = { + def onTaskStart(taskId: Long, stageId: Int, stageAttemptId: Int): Unit = { + // Put an entry in taskMetricPeaks for the task. + taskMetricPeaks.put(taskId, new AtomicLongArray(ExecutorMetricType.numMetrics)) + // Put a new entry in stageTCMP for the stage if there isn't one already. // Increment the task count. val (count, _) = stageTCMP.computeIfAbsent((stageId, stageAttemptId), @@ -123,14 +124,16 @@ private[spark] class ExecutorMetricsPoller( } /** - * Called by TaskRunner#run. + * Called by TaskRunner#run. It should only be called if onTaskStart has been called with + * the same arguments. * + * @param taskId the id of the task that was run. * @param stageId the id of the stage the task belongs to. * @param stageAttemptId the attempt number of the stage the task belongs to. */ - def onTaskCompletion(stageId: Int, stageAttemptId: Int): Unit = { + def onTaskCompletion(taskId: Long, stageId: Int, stageAttemptId: Int): Unit = { // Decrement the task count. - // Remove the entry from stageTCMP if the task count reaches zero + // Remove the entry from stageTCMP if the task count reaches zero. def decrementCount(stage: StageKey, countAndPeaks: TCMP): TCMP = { val count = countAndPeaks._1 @@ -145,14 +148,8 @@ private[spark] class ExecutorMetricsPoller( } stageTCMP.computeIfPresent((stageId, stageAttemptId), decrementCount) - } - /** - * Called by TaskRunner#run. - * - * @param taskId the id of the task that was run. - */ - def onTaskCleanup(taskId: Long): Unit = { + // Remove the entry from taskMetricPeaks for the task. taskMetricPeaks.remove(taskId) } @@ -180,16 +177,7 @@ private[spark] class ExecutorMetricsPoller( def getExecutorUpdates(): HashMap[StageKey, ExecutorMetrics] = { // build the executor level memory metrics val executorUpdates = new HashMap[StageKey, ExecutorMetrics] - - def peaksForStage(k: StageKey, v: TCMP): (StageKey, AtomicLongArray) = - if (v._1.get() > 0) (k, v._2) else null - - def addPeaks(nested: (StageKey, AtomicLongArray)): Unit = { - val (k, v) = nested - executorUpdates.put(k, new ExecutorMetrics(v)) - } - - stageTCMP.forEach[(StageKey, AtomicLongArray)](LONG_MAX_VALUE, peaksForStage, addPeaks) + stageTCMP.forEach((k, v) => executorUpdates.put(k, new ExecutorMetrics(v._2))) // reset the peaks def resetPeaks(k: StageKey, v: TCMP): TCMP = diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 289784979ff0..b6df216d537e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -78,11 +78,10 @@ private[spark] class DirectTaskResult[T]( if (numMetrics == 0) { metricPeaks = Array.empty } else { - val _metricPeaks = new ArrayBuffer[Long] - for (i <- 0 until numMetrics) { - _metricPeaks += in.readLong + metricPeaks = new Array[Long](numMetrics) + (0 until numMetrics).foreach { i => + metricPeaks(i) = in.readLong } - metricPeaks = _metricPeaks.toArray } valueObjectDeserialized = false } diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 366a8e63f829..2f3be298d3a8 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -341,12 +341,7 @@ class ExecutorSuite extends SparkFunSuite // Ensure that the executor's metricsPoller is polled at least once so that values are // recorded for the task metrics - val executorClass = classOf[Executor] - val metricsPollerField = - executorClass.getDeclaredField("org$apache$spark$executor$Executor$$metricsPoller") - metricsPollerField.setAccessible(true) - val metricsPoller = metricsPollerField.get(executor).asInstanceOf[ExecutorMetricsPoller] - metricsPoller.poll() + executor.metricsPoller.poll() eventually(timeout(1.seconds), interval(10.milliseconds)) { assert(executor.numRunningTasks === 0) } @@ -460,12 +455,7 @@ class ExecutorSuite extends SparkFunSuite // Ensure that the executor's metricsPoller is polled at least once so that values are // recorded for the task metrics. // When the task fails, peak values for these metrics get sent in the TaskFailedReason. - val executorClass = classOf[Executor] - val metricsPollerField = - executorClass.getDeclaredField("org$apache$spark$executor$Executor$$metricsPoller") - metricsPollerField.setAccessible(true) - val metricsPoller = metricsPollerField.get(executor).asInstanceOf[ExecutorMetricsPoller] - metricsPoller.poll() + executor.metricsPoller.poll() } if (killTask) { val killingThread = new Thread("kill-task") { 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 df1e79fec542..f11e0e9b4b76 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -475,7 +475,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(lines(1).contains("SparkListenerApplicationStart")) assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) var logIdx = 1 - events.foreach {event => + events.foreach { event => event match { case metricsUpdate: SparkListenerExecutorMetricsUpdate => case stageCompleted: SparkListenerStageCompleted => @@ -489,9 +489,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(execIds.size == 3) // check that each executor/driver was logged checkEvent(lines(logIdx), event) logIdx += 1 - case _ => - checkEvent(lines(logIdx), event) - logIdx += 1 + case _ => + checkEvent(lines(logIdx), event) + logIdx += 1 } } } finally { @@ -565,14 +565,14 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit (event, parsed) match { case (expected: SparkListenerStageSubmitted, actual: SparkListenerStageSubmitted) => // accumulables can be different, so only check the stage Id - assert(expected.stageInfo.stageId == actual.stageInfo.stageId) + assert(expected.stageInfo.stageId === actual.stageInfo.stageId) case (expected: SparkListenerStageCompleted, actual: SparkListenerStageCompleted) => // accumulables can be different, so only check the stage Id - assert(expected.stageInfo.stageId == actual.stageInfo.stageId) + assert(expected.stageInfo.stageId === actual.stageInfo.stageId) case (expected: SparkListenerTaskEnd, actual: SparkListenerTaskEnd) => // taskInfo, taskExecutorMetrics, and taskMetrics will be different object references, // so only check stageId - assert(expected.stageId == actual.stageId) + assert(expected.stageId === actual.stageId) case (expected: SparkListenerEvent, actual: SparkListenerEvent) => assert(expected === actual) } From 3ed583ad767f08efd50e2754082a1d6a3b6ffd21 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 27 Mar 2019 17:24:50 -0700 Subject: [PATCH 12/22] [SPARK-26329][CORE] Fix ExecutorSuite failures. Add some explanatory comments around existing helper methods and classes and how they are currently invoked. --- .../executor/ExecutorMetricsPoller.scala | 14 ++-- .../apache/spark/executor/ExecutorSuite.scala | 69 ++++++++++++++----- 2 files changed, 61 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala index a6a98c1e8654..5813d741d3b4 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -107,7 +107,7 @@ private[spark] class ExecutorMetricsPoller( /** * Called by TaskRunner#run. * - * @param taskId the id of the task being launched. + * @param taskId the id of the task being run. * @param stageId the id of the stage the task belongs to. * @param stageAttemptId the attempt number of the stage the task belongs to. */ @@ -159,10 +159,14 @@ private[spark] class ExecutorMetricsPoller( * @param taskId the id of the task that was run. */ def getTaskMetricPeaks(taskId: Long): Array[Long] = { - val currentPeaks = taskMetricPeaks.get(taskId) - val metricPeaks = new Array[Long](ExecutorMetricType.numMetrics) - ExecutorMetricType.metricToOffset.foreach { case (_, i) => - metricPeaks(i) = currentPeaks.get(i) + // If this is called with an invalid taskId or a valid taskId but the task was killed and + // onTaskStart was therefore not called, then we return an array of zeros. + val currentPeaks = taskMetricPeaks.get(taskId) // may be null + val metricPeaks = new Array[Long](ExecutorMetricType.numMetrics) // initialized to zeros + if (currentPeaks != null) { + ExecutorMetricType.metricToOffset.foreach { case (_, i) => + metricPeaks(i) = currentPeaks.get(i) + } } metricPeaks } diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 2f3be298d3a8..eb5220e8bfe5 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -136,6 +136,10 @@ class ExecutorSuite extends SparkFunSuite } } + // This test does not use ExecutorSuiteHelper.latches. + // It instantiates a FetchFailureHidingRDD with throwOOM = false and interrupt = false. + // It calls runTaskGetFailReasonAndExceptionHandler (via runTaskAndGetFailReason) with + // killTask = false and poll = false. test("SPARK-19276: Handle FetchFailedExceptions that are hidden by user exceptions") { val conf = new SparkConf().setMaster("local").setAppName("executor suite test") sc = new SparkContext(conf) @@ -203,18 +207,18 @@ class ExecutorSuite extends SparkFunSuite // Submit a job where a fetch failure is thrown, but then there is an OOM or interrupt. We // should treat the fetch failure as a false positive, and do normal OOM or interrupt handling. val inputRDD = new FetchFailureThrowingRDD(sc) - if (!oom) { - // we are trying to setup a case where a task is killed after a fetch failure -- this - // is just a helper to coordinate between the task thread and this thread that will - // kill the task - ExecutorSuiteHelper.latches = new ExecutorSuiteHelper() - } + // helper to coordinate between the task thread and this thread that will kill the task + // (and to poll executor metrics if necessary) + ExecutorSuiteHelper.latches = new ExecutorSuiteHelper val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = oom, interrupt = !oom) val taskDescription = createResultTaskDescription(serializer, resultFunc, secondRDD, 1) runTaskGetFailReasonAndExceptionHandler(taskDescription, killTask = !oom, poll) } + // This test does not use ExecutorSuiteHelper.latches. + // It calls runTaskGetFailReasonAndExceptionHandler (via runTaskAndGetFailReason) with + // killTask = false and poll = false. test("Gracefully handle error in task deserialization") { val conf = new SparkConf val serializer = new JavaSerializer(conf) @@ -318,10 +322,19 @@ class ExecutorSuite extends SparkFunSuite test("Send task executor metrics in DirectTaskResult") { // Run a successful, trivial result task + // We need to ensure, however, that executor metrics are polled after the task is started + // so this requires some coordination using ExecutorSuiteHelper. val conf = new SparkConf().setMaster("local").setAppName("executor suite test") sc = new SparkContext(conf) val serializer = SparkEnv.get.closureSerializer.newInstance() - val resultFunc = (context: TaskContext, itr: Iterator[Int]) => itr.size + ExecutorSuiteHelper.latches = new ExecutorSuiteHelper + val resultFunc = + (context: TaskContext, itr: Iterator[Int]) => { + ExecutorSuiteHelper.latches.latch1.await(300, TimeUnit.MILLISECONDS) + ExecutorSuiteHelper.latches.latch2.countDown() + ExecutorSuiteHelper.latches.latch3.await(500, TimeUnit.MILLISECONDS) + itr.size + } val rdd = new RDD[Int](sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[Int] = { val l = List(1) @@ -334,14 +347,22 @@ class ExecutorSuite extends SparkFunSuite val taskDescription = createResultTaskDescription(serializer, resultFunc, rdd, 0) val mockBackend = mock[ExecutorBackend] + when(mockBackend.statusUpdate(any(), meq(TaskState.RUNNING), any())) + .thenAnswer(new Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + ExecutorSuiteHelper.latches.latch1.countDown() + } + }) var executor: Executor = null try { executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true) executor.launchTask(mockBackend, taskDescription) - // Ensure that the executor's metricsPoller is polled at least once so that values are - // recorded for the task metrics + // Ensure that the executor's metricsPoller is polled so that values are recorded for + // the task metrics + ExecutorSuiteHelper.latches.latch2.await(500, TimeUnit.MILLISECONDS) executor.metricsPoller.poll() + ExecutorSuiteHelper.latches.latch3.countDown() eventually(timeout(1.seconds), interval(10.milliseconds)) { assert(executor.numRunningTasks === 0) } @@ -438,6 +459,8 @@ class ExecutorSuite extends SparkFunSuite runTaskGetFailReasonAndExceptionHandler(taskDescription, false)._1 } + // NOTE: This method is ever only called with killTask = false and poll = false when we are + // not using ExecutorSuiteHelper.latches. private def runTaskGetFailReasonAndExceptionHandler( taskDescription: TaskDescription, killTask: Boolean, @@ -451,18 +474,16 @@ class ExecutorSuite extends SparkFunSuite uncaughtExceptionHandler = mockUncaughtExceptionHandler) // the task will be launched in a dedicated worker thread executor.launchTask(mockBackend, taskDescription) - if (poll) { - // Ensure that the executor's metricsPoller is polled at least once so that values are - // recorded for the task metrics. - // When the task fails, peak values for these metrics get sent in the TaskFailedReason. - executor.metricsPoller.poll() - } if (killTask) { val killingThread = new Thread("kill-task") { override def run(): Unit = { // wait to kill the task until it has thrown a fetch failure if (ExecutorSuiteHelper.latches.latch1.await(10, TimeUnit.SECONDS)) { // now we can kill the task + // but before that, ensure that the executor's metricsPoller is polled + if (poll) { + executor.metricsPoller.poll() + } executor.killAllTasks(true, "Killed task, eg. because of speculative execution") } else { timedOut.set(true) @@ -470,6 +491,14 @@ class ExecutorSuite extends SparkFunSuite } } killingThread.start() + } else { + // As noted above, when killTask = false and poll = false, we are not using latches; + // thus we only need to wait for latch1 and countdown latch2 when poll = true. + if (poll) { + ExecutorSuiteHelper.latches.latch1.await(1, TimeUnit.SECONDS) + executor.metricsPoller.poll() + ExecutorSuiteHelper.latches.latch2.countDown() + } } eventually(timeout(5.seconds), interval(10.milliseconds)) { assert(executor.numRunningTasks === 0) @@ -521,6 +550,8 @@ class SimplePartition extends Partition { override def index: Int = 0 } +// NOTE: When instantiating this class, except with throwOOM = false and interrupt = false, +// ExecutorSuiteHelper.latches need to be set (not null). class FetchFailureHidingRDD( sc: SparkContext, val input: FetchFailureThrowingRDD, @@ -533,13 +564,17 @@ class FetchFailureHidingRDD( } catch { case t: Throwable => if (throwOOM) { + // Allow executor metrics to be polled (if necessary) before throwing the OOMError + ExecutorSuiteHelper.latches.latch1.countDown() + ExecutorSuiteHelper.latches.latch2.await(500, TimeUnit.MILLISECONDS) // scalastyle:off throwerror throw new OutOfMemoryError("OOM while handling another exception") // scalastyle:on throwerror } else if (interrupt) { // make sure our test is setup correctly assert(TaskContext.get().asInstanceOf[TaskContextImpl].fetchFailed.isDefined) - // signal our test is ready for the task to get killed + // signal we are ready for executor metrics to be polled (if necessary) and for + // the task to get killed ExecutorSuiteHelper.latches.latch1.countDown() // then wait for another thread in the test to kill the task -- this latch // is never actually decremented, we just wait to get killed. @@ -567,7 +602,7 @@ private class ExecutorSuiteHelper { @volatile var testFailedReason: TaskFailedReason = _ } -// helper for coordinating killing tasks +// Helper for coordinating killing tasks as well as polling executor metrics private object ExecutorSuiteHelper { var latches: ExecutorSuiteHelper = null } From 0a4828a91ff865913d0553d2aba6a19b6ba5c834 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Thu, 28 Mar 2019 11:15:15 -0700 Subject: [PATCH 13/22] [SPARK-26329][CORE] Delete a comment on irashid's suggestion. --- .../org/apache/spark/scheduler/EventLoggingListenerSuite.scala | 2 -- 1 file changed, 2 deletions(-) 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 f11e0e9b4b76..a83ca594ee90 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -570,8 +570,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // accumulables can be different, so only check the stage Id assert(expected.stageInfo.stageId === actual.stageInfo.stageId) case (expected: SparkListenerTaskEnd, actual: SparkListenerTaskEnd) => - // taskInfo, taskExecutorMetrics, and taskMetrics will be different object references, - // so only check stageId assert(expected.stageId === actual.stageId) case (expected: SparkListenerEvent, actual: SparkListenerEvent) => assert(expected === actual) From 9530b75eb23476341222db6d87b3bac447d94598 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Thu, 28 Mar 2019 11:17:01 -0700 Subject: [PATCH 14/22] [SPARK-26329][CORE] Change executorUpdates to be a scala.collection.mutable.Map. In Heartbeat, since the Map we actually construct and send as executorUpdates is a mutable one, make the signature use a scala.collection,mutable.Map instead of scala.collection.Map. --- .../main/scala/org/apache/spark/HeartbeatReceiver.scala | 7 +++---- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 5 +++-- .../scala/org/apache/spark/scheduler/TaskScheduler.scala | 2 +- .../scala/org/apache/spark/HeartbeatReceiverSuite.scala | 2 +- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 4 ++-- .../spark/scheduler/ExternalClusterManagerSuite.scala | 2 +- 6 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 4e6679f5abcf..20224eb721c0 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -19,8 +19,7 @@ package org.apache.spark import java.util.concurrent.{ScheduledFuture, TimeUnit} -import scala.collection.Map -import scala.collection.mutable.HashMap +import scala.collection.mutable.{HashMap, Map} import scala.concurrent.Future import org.apache.spark.executor.ExecutorMetrics @@ -123,14 +122,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) context.reply(true) // Messages received from executors - case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId, executorMetrics) => + case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId, executorUpdates) => 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, executorMetrics) + executorId, accumUpdates, blockManagerId, executorUpdates) val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) context.reply(response) } 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 afb4c88be5e5..482691c94f87 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.annotation.tailrec import scala.collection.Map +import scala.collection.mutable import scala.collection.mutable.{HashMap, HashSet, ListBuffer} import scala.concurrent.duration._ import scala.util.control.NonFatal @@ -265,7 +266,7 @@ private[spark] class DAGScheduler( accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId, // (stageId, stageAttemptId) -> metrics - executorUpdates: Map[(Int, Int), ExecutorMetrics]): Boolean = { + executorUpdates: mutable.Map[(Int, Int), ExecutorMetrics]): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, executorUpdates)) blockManagerMaster.driverEndpoint.askSync[Boolean]( @@ -1576,7 +1577,7 @@ private[spark] class DAGScheduler( // in the stage chains that connect to the `failedStage`. To speed up the stage // traversing, we collect the stages to rollback first. If a stage needs to // rollback, all its succeeding stages need to rollback to. - val stagesToRollback = scala.collection.mutable.HashSet(failedStage) + val stagesToRollback = HashSet(failedStage) def collectStagesToRollback(stageChain: List[Stage]): Unit = { if (stagesToRollback.contains(stageChain.head)) { 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 30c8f405317b..8c73d563043c 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,7 @@ package org.apache.spark.scheduler -import scala.collection.Map +import scala.collection.mutable.Map import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.scheduler.SchedulingMode.SchedulingMode diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index bb092a1251d5..dfe33b1e5269 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -217,7 +217,7 @@ class HeartbeatReceiverSuite val blockManagerId = BlockManagerId(executorId, "localhost", 12345) val executorMetrics = new ExecutorMetrics(Array(123456L, 543L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) - val executorUpdates = Map((0, 0) -> executorMetrics) + val executorUpdates = mutable.Map((0, 0) -> executorMetrics) 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 0d5fe82651c7..ed7e43a9ae86 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -144,7 +144,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorUpdates: scala.collection.Map[(Int, Int), ExecutorMetrics]): Boolean = true + executorUpdates: Map[(Int, Int), ExecutorMetrics]): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -676,7 +676,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi execId: String, accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, - executorMetrics: scala.collection.Map[(Int, Int), ExecutorMetrics]): Boolean = true + executorUpdates: Map[(Int, Int), 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/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index f38dd2cba025..73e88c4a0fda 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import scala.collection.Map +import scala.collection.mutable.Map import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.executor.ExecutorMetrics From 38a397cd502fd1d2545dffd1b8cb6fb2399737d1 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Tue, 9 Apr 2019 16:10:14 -0700 Subject: [PATCH 15/22] [SPARK-26329][CORE] Update HistoryServerSuite. There are three existing cases in HistoryServerSuite related to executor metrics. Replace them with a single case that shows all the executor metrics we currently collect; they include proc fs metrics and garbage collection netrics. Uploaded new event logs. These show executor metrics at task end. --- .../application_list_json_expectation.json | 53 +-- .../completed_app_list_json_expectation.json | 51 +-- ...e_collection_metrics_json_expectation.json | 122 ------ ...ith_executor_metrics_json_expectation.json | 402 ++++++------------ ...process_tree_metrics_json_expectation.json | 108 ----- .../limit_app_list_json_expectation.json | 65 ++- .../minDate_app_list_json_expectation.json | 53 +-- .../minEndDate_app_list_json_expectation.json | 53 +-- .../application_1506645932520_24630151 | 63 --- .../application_1536831636016_59384_1 | 214 ---------- .../application_1538416563558_0014 | 190 --------- .../application_1553914137147_0018 | 44 ++ .../deploy/history/HistoryServerSuite.scala | 6 +- dev/.rat-excludes | 4 +- 14 files changed, 255 insertions(+), 1173 deletions(-) delete mode 100644 core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json delete mode 100644 core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json delete mode 100644 core/src/test/resources/spark-events/application_1506645932520_24630151 delete mode 100755 core/src/test/resources/spark-events/application_1536831636016_59384_1 delete mode 100644 core/src/test/resources/spark-events/application_1538416563558_0014 create mode 100644 core/src/test/resources/spark-events/application_1553914137147_0018 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 71ccedced788..503f6101535b 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,50 +1,17 @@ [ { - "id": "application_1536831636016_59384", - "name": "Spark Pi", - "attempts": [ - { - "attemptId": "1", - "startTime": "2019-01-08T04:33:43.607GMT", - "endTime": "2019-01-08T04:33:58.745GMT", - "lastUpdated": "", - "duration": 15138, - "sparkUser": "lajin", - "completed": true, - "appSparkVersion": "3.0.0-SNAPSHOT", - "lastUpdatedEpoch": 0, - "startTimeEpoch": 1546922023607, - "endTimeEpoch": 1546922038745 - } - ] -}, { - "id" : "application_1538416563558_0014", - "name" : "PythonBisectingKMeansExample", + "id" : "application_1553914137147_0018", + "name" : "LargeBlocks", "attempts" : [ { - "startTime" : "2018-10-02T00:42:39.580GMT", - "endTime" : "2018-10-02T00:44:02.338GMT", + "startTime" : "2019-04-08T20:39:44.286GMT", + "endTime" : "2019-04-08T20:40:46.454GMT", "lastUpdated" : "", - "duration" : 82758, - "sparkUser" : "root", + "duration" : 62168, + "sparkUser" : "systest", "completed" : true, - "appSparkVersion" : "2.5.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1538440959580, - "endTimeEpoch" : 1538441042338 - } ] -}, { - "id" : "application_1506645932520_24630151", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-04-19T23:54:42.734GMT", - "endTime" : "2018-04-19T23:56:29.134GMT", - "lastUpdated" : "", - "duration" : 106400, - "sparkUser" : "edlu", - "completed" : true, - "appSparkVersion" : "2.4.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1524182082734, - "endTimeEpoch" : 1524182189134 + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1554755984286, + "endTimeEpoch" : 1554756046454, + "lastUpdatedEpoch" : 0 } ] }, { "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 ad5f0ea9699d..503f6101535b 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,49 +1,16 @@ [ { - "id": "application_1536831636016_59384", - "name": "Spark Pi", - "attempts": [ - { - "attemptId": "1", - "startTime": "2019-01-08T04:33:43.607GMT", - "endTime": "2019-01-08T04:33:58.745GMT", - "lastUpdated": "", - "duration": 15138, - "sparkUser": "lajin", - "completed": true, - "appSparkVersion": "3.0.0-SNAPSHOT", - "lastUpdatedEpoch": 0, - "startTimeEpoch": 1546922023607, - "endTimeEpoch": 1546922038745 - } - ] -}, { - "id" : "application_1538416563558_0014", - "name" : "PythonBisectingKMeansExample", - "attempts" : [ { - "startTime" : "2018-10-02T00:42:39.580GMT", - "endTime" : "2018-10-02T00:44:02.338GMT", - "lastUpdated" : "", - "duration" : 82758, - "sparkUser" : "root", - "completed" : true, - "appSparkVersion" : "2.5.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1538440959580, - "endTimeEpoch" : 1538441042338 - } ] -}, { - "id" : "application_1506645932520_24630151", - "name" : "Spark shell", + "id" : "application_1553914137147_0018", + "name" : "LargeBlocks", "attempts" : [ { - "startTime" : "2018-04-19T23:54:42.734GMT", - "endTime" : "2018-04-19T23:56:29.134GMT", + "startTime" : "2019-04-08T20:39:44.286GMT", + "endTime" : "2019-04-08T20:40:46.454GMT", "lastUpdated" : "", - "duration" : 106400, - "sparkUser" : "edlu", + "duration" : 62168, + "sparkUser" : "systest", "completed" : true, - "appSparkVersion" : "2.4.0-SNAPSHOT", - "startTimeEpoch" : 1524182082734, - "endTimeEpoch" : 1524182189134, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1554755984286, + "endTimeEpoch" : 1554756046454, "lastUpdatedEpoch" : 0 } ] }, { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json deleted file mode 100644 index f0f39e60bdd5..000000000000 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json +++ /dev/null @@ -1,122 +0,0 @@ -[ { - "id" : "driver", - "hostPort" : "047.company.com:42509", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 0, - "maxTasks" : 0, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 0, - "totalTasks" : 0, - "totalDuration" : 0, - "totalGCTime" : 0, - "totalInputBytes" : 0, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : false, - "maxMemory" : 100977868, - "addTime" : "2019-01-08T04:33:44.502GMT", - "executorLogs" : { - "stdout" : "https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stdout?start=-4096", - "stderr" : "https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 100977868, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "JVMHeapMemory" : 211171816, - "JVMOffHeapMemory" : 90237256, - "OnHeapExecutionMemory" : 0, - "OffHeapExecutionMemory" : 0, - "OnHeapStorageMemory" : 4876, - "OffHeapStorageMemory" : 0, - "OnHeapUnifiedMemory" : 4876, - "OffHeapUnifiedMemory" : 0, - "DirectPoolMemory" : 806275, - "MappedPoolMemory" : 0, - "ProcessTreeJVMVMemory" : 2646888448, - "ProcessTreeJVMRSSMemory" : 520900608, - "ProcessTreePythonVMemory" : 0, - "ProcessTreePythonRSSMemory" : 0, - "ProcessTreeOtherVMemory" : 0, - "ProcessTreeOtherRSSMemory" : 0, - "MinorGCCount" : 8, - "MinorGCTime" : 374, - "MajorGCCount" : 3, - "MajorGCTime" : 170 - }, - "attributes" : { } -}, { - "id" : "2", - "hostPort" : "028.company.com:46325", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 4, - "maxTasks" : 4, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 52, - "totalTasks" : 52, - "totalDuration" : 8879, - "totalGCTime" : 420, - "totalInputBytes" : 0, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : false, - "maxMemory" : 97832140, - "addTime" : "2019-01-08T04:33:54.270GMT", - "executorLogs" : { - "stdout" : "https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stdout?start=-4096", - "stderr" : "https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 97832140, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "attributes" : { } -}, { - "id" : "1", - "hostPort" : "036.company.com:35126", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 4, - "maxTasks" : 4, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 48, - "totalTasks" : 48, - "totalDuration" : 8837, - "totalGCTime" : 1192, - "totalInputBytes" : 0, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : false, - "maxMemory" : 97832140, - "addTime" : "2019-01-08T04:33:55.929GMT", - "executorLogs" : { - "stdout" : "https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stdout?start=-4096", - "stderr" : "https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 97832140, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "attributes" : { } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json index 3db537799b29..5b9174a04d83 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" : "node0033.grid.company.com:60749", + "hostPort" : "test-1.vpc.company.com:44768", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -17,163 +17,9 @@ "totalShuffleRead" : 0, "totalShuffleWrite" : 0, "isBlacklisted" : false, - "maxMemory" : 1043437977, - "addTime" : "2018-04-19T23:55:05.107GMT", - "executorLogs" : { }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 1043437977, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "OnHeapStorageMemory" : 905801, - "JVMOffHeapMemory" : 205304696, - "OffHeapExecutionMemory" : 0, - "OnHeapUnifiedMemory" : 905801, - "OnHeapExecutionMemory" : 0, - "OffHeapUnifiedMemory" : 0, - "DirectPoolMemory" : 397602, - "MappedPoolMemory" : 0, - "JVMHeapMemory" : 629553808, - "OffHeapStorageMemory" : 0, - "ProcessTreeJVMVMemory": 0, - "ProcessTreeJVMRSSMemory": 0, - "ProcessTreePythonVMemory": 0, - "ProcessTreePythonRSSMemory": 0, - "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0, - "MinorGCCount": 0, - "MinorGCTime": 0, - "MajorGCCount": 0, - "MajorGCTime": 0 - }, - "attributes" : { } -}, { - "id" : "7", - "hostPort" : "node6340.grid.company.com:5933", - "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-04-19T23:55:49.826GMT", - "executorLogs" : { - "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, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 956615884, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "attributes" : { } -}, { - "id" : "6", - "hostPort" : "node6644.grid.company.com:8445", - "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-04-19T23:55:47.549GMT", - "executorLogs" : { - "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, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 956615884, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "attributes" : { } -}, { - "id" : "5", - "hostPort" : "node2477.grid.company.com:20123", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 1, - "maxTasks" : 1, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 9252, - "totalGCTime" : 920, - "totalInputBytes" : 36838295, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 355051, - "isBlacklisted" : false, - "maxMemory" : 956615884, - "addTime" : "2018-04-19T23:55:43.160GMT", - "executorLogs" : { - "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, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 956615884, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "attributes" : { } -}, { - "id" : "4", - "hostPort" : "node4243.grid.company.com:16084", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 1, - "maxTasks" : 1, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 3, - "totalTasks" : 3, - "totalDuration" : 15645, - "totalGCTime" : 405, - "totalInputBytes" : 87272855, - "totalShuffleRead" : 438675, - "totalShuffleWrite" : 26773039, - "isBlacklisted" : false, "maxMemory" : 956615884, - "addTime" : "2018-04-19T23:55:12.278GMT", - "executorLogs" : { - "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" - }, + "addTime" : "2019-04-08T20:39:49.747GMT", + "executorLogs" : { }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, "usedOffHeapStorageMemory" : 0, @@ -182,31 +28,31 @@ }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "OnHeapStorageMemory" : 63104457, - "JVMOffHeapMemory" : 95657456, + "JVMHeapMemory" : 209883992, + "JVMOffHeapMemory" : 91926448, + "OnHeapExecutionMemory" : 0, "OffHeapExecutionMemory" : 0, - "OnHeapUnifiedMemory" : 100853193, - "OnHeapExecutionMemory" : 37748736, + "OnHeapStorageMemory" : 19560, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 19560, "OffHeapUnifiedMemory" : 0, - "DirectPoolMemory" : 126261, + "DirectPoolMemory" : 157182, "MappedPoolMemory" : 0, - "JVMHeapMemory" : 518613056, - "OffHeapStorageMemory" : 0, - "ProcessTreeJVMVMemory": 0, - "ProcessTreeJVMRSSMemory": 0, - "ProcessTreePythonVMemory": 0, - "ProcessTreePythonRSSMemory": 0, - "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0, - "MinorGCCount": 0, - "MinorGCTime": 0, - "MajorGCCount": 0, - "MajorGCTime": 0 + "ProcessTreeJVMVMemory" : 8296026112, + "ProcessTreeJVMRSSMemory" : 526491648, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 7, + "MinorGCTime" : 55, + "MajorGCCount" : 3, + "MajorGCTime" : 144 }, "attributes" : { } }, { "id" : "3", - "hostPort" : "node0998.grid.company.com:45265", + "hostPort" : "test-3.vpc.company.com:37641", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -215,53 +61,63 @@ "maxTasks" : 1, "activeTasks" : 0, "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 14491, - "totalGCTime" : 342, - "totalInputBytes" : 50409514, + "completedTasks" : 2, + "totalTasks" : 2, + "totalDuration" : 31015, + "totalGCTime" : 151, + "totalInputBytes" : 2500050000, "totalShuffleRead" : 0, - "totalShuffleWrite" : 31362123, + "totalShuffleWrite" : 0, "isBlacklisted" : false, - "maxMemory" : 956615884, - "addTime" : "2018-04-19T23:55:12.088GMT", + "maxMemory" : 3820172083, + "addTime" : "2019-04-08T20:39:55.362GMT", "executorLogs" : { - "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" + "stdout" : "http://test-3.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000004/systest/stdout?start=-4096", + "stderr" : "http://test-3.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000004/systest/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 956615884, + "totalOnHeapStorageMemory" : 3820172083, "totalOffHeapStorageMemory" : 0 }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "OnHeapStorageMemory" : 69535048, - "JVMOffHeapMemory" : 90709624, - "OffHeapExecutionMemory" : 0, - "OnHeapUnifiedMemory" : 69535048, + "JVMHeapMemory" : 759479480, + "JVMOffHeapMemory" : 61343616, "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 12537, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 12537, "OffHeapUnifiedMemory" : 0, - "DirectPoolMemory" : 87796, + "DirectPoolMemory" : 10865, "MappedPoolMemory" : 0, - "JVMHeapMemory" : 726805712, - "OffHeapStorageMemory" : 0, - "ProcessTreeJVMVMemory": 0, - "ProcessTreeJVMRSSMemory": 0, - "ProcessTreePythonVMemory": 0, - "ProcessTreePythonRSSMemory": 0, - "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0, - "MinorGCCount": 0, - "MinorGCTime": 0, - "MajorGCCount": 0, - "MajorGCTime": 0 - }, - "attributes" : { } + "ProcessTreeJVMVMemory" : 9691553792, + "ProcessTreeJVMRSSMemory" : 964222976, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 24, + "MinorGCTime" : 145, + "MajorGCCount" : 2, + "MajorGCTime" : 63 + }, + "attributes" : { + "NM_HTTP_ADDRESS" : "test-3.vpc.company.com:8042", + "USER" : "systest", + "LOG_FILES" : "stderr,stdout", + "NM_HTTP_PORT" : "8042", + "CLUSTER_ID" : "", + "NM_PORT" : "8041", + "HTTP_SCHEME" : "http://", + "NM_HOST" : "test-3.vpc.company.com", + "CONTAINER_ID" : "container_1553914137147_0018_01_000004" + } }, { "id" : "2", - "hostPort" : "node4045.grid.company.com:29262", + "hostPort" : "test-4.vpc.company.com:33179", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -272,51 +128,61 @@ "failedTasks" : 0, "completedTasks" : 1, "totalTasks" : 1, - "totalDuration" : 14113, - "totalGCTime" : 326, - "totalInputBytes" : 50423423, + "totalDuration" : 30302, + "totalGCTime" : 121, + "totalInputBytes" : 2500050000, "totalShuffleRead" : 0, - "totalShuffleWrite" : 22950296, + "totalShuffleWrite" : 0, "isBlacklisted" : false, - "maxMemory" : 956615884, - "addTime" : "2018-04-19T23:55:12.471GMT", + "maxMemory" : 3820172083, + "addTime" : "2019-04-08T20:39:55.291GMT", "executorLogs" : { - "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" + "stdout" : "http://test-4.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000003/systest/stdout?start=-4096", + "stderr" : "http://test-4.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000003/systest/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 956615884, + "totalOnHeapStorageMemory" : 3820172083, "totalOffHeapStorageMemory" : 0 }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "OnHeapStorageMemory" : 58468944, - "JVMOffHeapMemory" : 91208368, - "OffHeapExecutionMemory" : 0, - "OnHeapUnifiedMemory" : 58468944, + "JVMHeapMemory" : 462135680, + "JVMOffHeapMemory" : 62717520, "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 7023, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 7023, "OffHeapUnifiedMemory" : 0, - "DirectPoolMemory" : 87796, + "DirectPoolMemory" : 12781, "MappedPoolMemory" : 0, - "JVMHeapMemory" : 595946552, - "OffHeapStorageMemory" : 0, - "ProcessTreeJVMVMemory": 0, - "ProcessTreeJVMRSSMemory": 0, - "ProcessTreePythonVMemory": 0, - "ProcessTreePythonRSSMemory": 0, - "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0, - "MinorGCCount": 0, - "MinorGCTime": 0, - "MajorGCCount": 0, - "MajorGCTime": 0 - }, - "attributes" : { } + "ProcessTreeJVMVMemory" : 9684373504, + "ProcessTreeJVMRSSMemory" : 703639552, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 15, + "MinorGCTime" : 106, + "MajorGCCount" : 2, + "MajorGCTime" : 75 + }, + "attributes" : { + "NM_HTTP_ADDRESS" : "test-4.vpc.company.com:8042", + "USER" : "systest", + "LOG_FILES" : "stderr,stdout", + "NM_HTTP_PORT" : "8042", + "CLUSTER_ID" : "", + "NM_PORT" : "8041", + "HTTP_SCHEME" : "http://", + "NM_HOST" : "test-4.vpc.company.com", + "CONTAINER_ID" : "container_1553914137147_0018_01_000003" + } }, { "id" : "1", - "hostPort" : "node1404.grid.company.com:34043", + "hostPort" : "test-2.vpc.company.com:43764", "isActive" : true, "rddBlocks" : 0, "memoryUsed" : 0, @@ -327,46 +193,56 @@ "failedTasks" : 0, "completedTasks" : 3, "totalTasks" : 3, - "totalDuration" : 15665, - "totalGCTime" : 471, - "totalInputBytes" : 98905018, + "totalDuration" : 44900, + "totalGCTime" : 147, + "totalInputBytes" : 5000100000, "totalShuffleRead" : 0, - "totalShuffleWrite" : 20594744, + "totalShuffleWrite" : 0, "isBlacklisted" : false, - "maxMemory" : 956615884, - "addTime" : "2018-04-19T23:55:11.695GMT", + "maxMemory" : 3820172083, + "addTime" : "2019-04-08T20:39:54.596GMT", "executorLogs" : { - "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" + "stdout" : "http://test-2.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000002/systest/stdout?start=-4096", + "stderr" : "http://test-2.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000002/systest/stderr?start=-4096" }, "memoryMetrics" : { "usedOnHeapStorageMemory" : 0, "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 956615884, + "totalOnHeapStorageMemory" : 3820172083, "totalOffHeapStorageMemory" : 0 }, "blacklistedInStages" : [ ], "peakMemoryMetrics" : { - "OnHeapStorageMemory" : 47962185, - "JVMOffHeapMemory" : 100519936, - "OffHeapExecutionMemory" : 0, - "OnHeapUnifiedMemory" : 47962185, + "JVMHeapMemory" : 984736560, + "JVMOffHeapMemory" : 60829472, "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 19560, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 19560, "OffHeapUnifiedMemory" : 0, - "DirectPoolMemory" : 98230, + "DirectPoolMemory" : 10742, "MappedPoolMemory" : 0, - "JVMHeapMemory" : 755008624, - "OffHeapStorageMemory" : 0, - "ProcessTreeJVMVMemory": 0, - "ProcessTreeJVMRSSMemory": 0, - "ProcessTreePythonVMemory": 0, - "ProcessTreePythonRSSMemory": 0, - "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0, - "MinorGCCount": 0, - "MinorGCTime": 0, - "MajorGCCount": 0, - "MajorGCTime": 0 - }, - "attributes" : { } + "ProcessTreeJVMVMemory" : 9678606336, + "ProcessTreeJVMRSSMemory" : 1210867712, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 27, + "MinorGCTime" : 140, + "MajorGCCount" : 2, + "MajorGCTime" : 60 + }, + "attributes" : { + "NM_HTTP_ADDRESS" : "test-2.vpc.company.com:8042", + "USER" : "systest", + "LOG_FILES" : "stderr,stdout", + "NM_HTTP_PORT" : "8042", + "CLUSTER_ID" : "", + "NM_PORT" : "8041", + "HTTP_SCHEME" : "http://", + "NM_HOST" : "test-2.vpc.company.com", + "CONTAINER_ID" : "container_1553914137147_0018_01_000002" + } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json deleted file mode 100644 index 2c2efb58341b..000000000000 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json +++ /dev/null @@ -1,108 +0,0 @@ -[ { - "id" : "driver", - "hostPort" : "rezamemory-1.gce.something.com:43959", - "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" : 384093388, - "addTime" : "2018-10-02T00:42:47.690GMT", - "executorLogs" : { }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 384093388, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "OnHeapStorageMemory" : 554933, - "JVMOffHeapMemory" : 104976128, - "OffHeapExecutionMemory" : 0, - "OnHeapUnifiedMemory" : 554933, - "OnHeapExecutionMemory" : 0, - "OffHeapUnifiedMemory" : 0, - "DirectPoolMemory" : 228407, - "MappedPoolMemory" : 0, - "JVMHeapMemory" : 350990264, - "OffHeapStorageMemory" : 0, - "ProcessTreeJVMVMemory" : 5067235328, - "ProcessTreeJVMRSSMemory" : 710475776, - "ProcessTreePythonVMemory" : 408375296, - "ProcessTreePythonRSSMemory" : 40284160, - "ProcessTreeOtherVMemory" : 0, - "ProcessTreeOtherRSSMemory" : 0, - "MinorGCCount": 0, - "MinorGCTime": 0, - "MajorGCCount": 0, - "MajorGCTime": 0 - }, - "attributes" : { } -}, { - "id" : "9", - "hostPort" : "rezamemory-2.gce.something.com:40797", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 1, - "maxTasks" : 1, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 2, - "totalTasks" : 2, - "totalDuration" : 6191, - "totalGCTime" : 288, - "totalInputBytes" : 108, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : false, - "maxMemory" : 384093388, - "addTime" : "2018-10-02T00:43:56.142GMT", - "executorLogs" : { - "stdout" : "http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stdout?start=-4096", - "stderr" : "http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 384093388, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "peakMemoryMetrics" : { - "OnHeapStorageMemory" : 1088805, - "JVMOffHeapMemory" : 59006656, - "OffHeapExecutionMemory" : 0, - "OnHeapUnifiedMemory" : 1088805, - "OnHeapExecutionMemory" : 0, - "OffHeapUnifiedMemory" : 0, - "DirectPoolMemory" : 20181, - "MappedPoolMemory" : 0, - "JVMHeapMemory" : 193766856, - "OffHeapStorageMemory" : 0, - "ProcessTreeJVMVMemory" : 3016261632, - "ProcessTreeJVMRSSMemory" : 405860352, - "ProcessTreePythonVMemory" : 625926144, - "ProcessTreePythonRSSMemory" : 69013504, - "ProcessTreeOtherVMemory" : 0, - "ProcessTreeOtherRSSMemory" : 0, - "MinorGCCount": 0, - "MinorGCTime": 0, - "MajorGCCount": 0, - "MajorGCTime": 0 - }, - "attributes" : { } -} ] diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index c303b6c565e3..62e068217c98 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,49 +1,46 @@ [ { - "id": "application_1536831636016_59384", - "name": "Spark Pi", - "attempts": [ - { - "attemptId": "1", - "startTime": "2019-01-08T04:33:43.607GMT", - "endTime": "2019-01-08T04:33:58.745GMT", - "lastUpdated": "", - "duration": 15138, - "sparkUser": "lajin", - "completed": true, - "appSparkVersion": "3.0.0-SNAPSHOT", - "lastUpdatedEpoch": 0, - "startTimeEpoch": 1546922023607, - "endTimeEpoch": 1546922038745 - } - ] + "id" : "application_1553914137147_0018", + "name" : "LargeBlocks", + "attempts" : [ { + "startTime" : "2019-04-08T20:39:44.286GMT", + "endTime" : "2019-04-08T20:40:46.454GMT", + "lastUpdated" : "", + "duration" : 62168, + "sparkUser" : "systest", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1554755984286, + "endTimeEpoch" : 1554756046454, + "lastUpdatedEpoch" : 0 + } ] }, { - "id" : "application_1538416563558_0014", - "name" : "PythonBisectingKMeansExample", + "id" : "application_1516285256255_0012", + "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-10-02T00:42:39.580GMT", - "endTime" : "2018-10-02T00:44:02.338GMT", + "startTime" : "2018-01-18T18:30:35.119GMT", + "endTime" : "2018-01-18T18:38:27.938GMT", "lastUpdated" : "", - "duration" : 82758, - "sparkUser" : "root", + "duration" : 472819, + "sparkUser" : "attilapiros", "completed" : true, - "appSparkVersion" : "2.5.0-SNAPSHOT", + "appSparkVersion" : "2.3.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1538440959580, - "endTimeEpoch" : 1538441042338 + "startTimeEpoch" : 1516300235119, + "endTimeEpoch" : 1516300707938 } ] }, { - "id" : "application_1506645932520_24630151", + "id" : "app-20180109111548-0000", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2018-04-19T23:54:42.734GMT", - "endTime" : "2018-04-19T23:56:29.134GMT", + "startTime" : "2018-01-09T10:15:42.372GMT", + "endTime" : "2018-01-09T10:24:37.606GMT", "lastUpdated" : "", - "duration" : 106400, - "sparkUser" : "edlu", + "duration" : 535234, + "sparkUser" : "attilapiros", "completed" : true, - "appSparkVersion" : "2.4.0-SNAPSHOT", + "appSparkVersion" : "2.3.0-SNAPSHOT", "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1524182082734, - "endTimeEpoch" : 1524182189134 + "startTimeEpoch" : 1515492942372, + "endTimeEpoch" : 1515493477606 } ] } ] 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 ba834d5f18d2..a212d544b37e 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,50 +1,17 @@ [ { - "id": "application_1536831636016_59384", - "name": "Spark Pi", - "attempts": [ - { - "attemptId": "1", - "startTime": "2019-01-08T04:33:43.607GMT", - "endTime": "2019-01-08T04:33:58.745GMT", - "lastUpdated": "", - "duration": 15138, - "sparkUser": "lajin", - "completed": true, - "appSparkVersion": "3.0.0-SNAPSHOT", - "lastUpdatedEpoch": 0, - "startTimeEpoch": 1546922023607, - "endTimeEpoch": 1546922038745 - } - ] -}, { - "id" : "application_1538416563558_0014", - "name" : "PythonBisectingKMeansExample", + "id" : "application_1553914137147_0018", + "name" : "LargeBlocks", "attempts" : [ { - "startTime" : "2018-10-02T00:42:39.580GMT", - "endTime" : "2018-10-02T00:44:02.338GMT", + "startTime" : "2019-04-08T20:39:44.286GMT", + "endTime" : "2019-04-08T20:40:46.454GMT", "lastUpdated" : "", - "duration" : 82758, - "sparkUser" : "root", + "duration" : 62168, + "sparkUser" : "systest", "completed" : true, - "appSparkVersion" : "2.5.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1538440959580, - "endTimeEpoch" : 1538441042338 - } ] -}, { - "id" : "application_1506645932520_24630151", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-04-19T23:54:42.734GMT", - "endTime" : "2018-04-19T23:56:29.134GMT", - "lastUpdated" : "", - "duration" : 106400, - "sparkUser" : "edlu", - "completed" : true, - "appSparkVersion" : "2.4.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1524182082734, - "endTimeEpoch" : 1524182189134 + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1554755984286, + "endTimeEpoch" : 1554756046454, + "lastUpdatedEpoch" : 0 } ] }, { "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 62d3544e0ba1..023e1ed496c3 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,50 +1,17 @@ [ { - "id": "application_1536831636016_59384", - "name": "Spark Pi", - "attempts": [ - { - "attemptId": "1", - "startTime": "2019-01-08T04:33:43.607GMT", - "endTime": "2019-01-08T04:33:58.745GMT", - "lastUpdated": "", - "duration": 15138, - "sparkUser": "lajin", - "completed": true, - "appSparkVersion": "3.0.0-SNAPSHOT", - "lastUpdatedEpoch": 0, - "startTimeEpoch": 1546922023607, - "endTimeEpoch": 1546922038745 - } - ] -}, { - "id" : "application_1538416563558_0014", - "name" : "PythonBisectingKMeansExample", + "id" : "application_1553914137147_0018", + "name" : "LargeBlocks", "attempts" : [ { - "startTime" : "2018-10-02T00:42:39.580GMT", - "endTime" : "2018-10-02T00:44:02.338GMT", + "startTime" : "2019-04-08T20:39:44.286GMT", + "endTime" : "2019-04-08T20:40:46.454GMT", "lastUpdated" : "", - "duration" : 82758, - "sparkUser" : "root", + "duration" : 62168, + "sparkUser" : "systest", "completed" : true, - "appSparkVersion" : "2.5.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1538440959580, - "endTimeEpoch" : 1538441042338 - } ] -}, { - "id" : "application_1506645932520_24630151", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-04-19T23:54:42.734GMT", - "endTime" : "2018-04-19T23:56:29.134GMT", - "lastUpdated" : "", - "duration" : 106400, - "sparkUser" : "edlu", - "completed" : true, - "appSparkVersion" : "2.4.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1524182082734, - "endTimeEpoch" : 1524182189134 + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1554755984286, + "endTimeEpoch" : 1554756046454, + "lastUpdatedEpoch" : 0 } ] }, { "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 deleted file mode 100644 index c48ed741c56e..000000000000 --- a/core/src/test/resources/spark-events/application_1506645932520_24630151 +++ /dev/null @@ -1,63 +0,0 @@ -{"Event":"SparkListenerLogStart","Spark Version":"2.4.0-SNAPSHOT"} -{"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":"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":[]}} -{"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":"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":[]}} -{"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":1524182189134} diff --git a/core/src/test/resources/spark-events/application_1536831636016_59384_1 b/core/src/test/resources/spark-events/application_1536831636016_59384_1 deleted file mode 100755 index 69924afead19..000000000000 --- a/core/src/test/resources/spark-events/application_1536831636016_59384_1 +++ /dev/null @@ -1,214 +0,0 @@ -{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"047.company.com","Port":42509},"Maximum Memory":100977868,"Timestamp":1546922024502,"Maximum Onheap Memory":100977868,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/apache/releases/jdk1.8.0_121/jre","Java Version":"1.8.0_121 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"/etc/security/keytabs/spark.service.keytab","spark.executor.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.driver.host":"047.company.com","spark.history.fs.logDirectory":"hdfs://hercules-sub/spark-logs","spark.eventLog.enabled":"true","spark.ssl.historyServer.trustStore":"/etc/hadoop/truststore.jks","spark.ui.port":"0","spark.driver.port":"36796","spark.shuffle.service.enabled":"true","spark.driver.extraLibraryPath":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native","spark.yarn.queue":"default","spark.history.fs.update.interval":"20s","spark.yarn.historyServer.address":"master2.company.com:50070","spark.yarn.app.id":"application_1536831636016_59384","spark.sql.function.eltOutputAsString":"true","spark.yarn.access.namenodes":"hdfs://hercules-sub,hdfs://hercules","spark.app.name":"Spark Pi","spark.scheduler.mode":"FIFO","spark.history.fs.numReplayThreads":"30","spark.eventLog.logStageExecutorMetrics.enabled":"true","spark.driver.memory":"512m","spark.executor.instances":"2","spark.history.kerberos.principal":"spark/master2.company.com@COMPANY.COM","spark.yarn.am.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.eventLog.logStageExecutorGCMetrics.enabled":"true","spark.ssl.historyServer.port":"49670","spark.ssl.historyServer.keyStoreType":"JKS","spark.sql.function.concatBinaryAsString":"true","spark.history.fs.cleaner.enabled":"true","spark.executor.id":"driver","spark.yarn.am.memory":"4g","spark.yarn.app.container.log.dir":"/hadoop/12/yarn/log/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001","spark.driver.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.sql.hive.caseSensitiveInferenceMode":"NEVER_INFER","spark.submit.deployMode":"cluster","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.RM_HA_URLS":"master1.company.com:50030,master2.company.com:50030","spark.master":"yarn","spark.ssl.historyServer.keyStorePassword":"*********(redacted)","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.extraLibraryPath":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native","spark.executor.memory":"512m","spark.driver.extraClassPath":"/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar:/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar","spark.eventLog.dir":"hdfs://hercules-sub/spark-logs","spark.ssl.historyServer.enabled":"true","spark.ssl.historyServer.keyStore":"/etc/hadoop/keystore.jks","spark.dynamicAllocation.enabled":"false","spark.executor.extraClassPath":"/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar:/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar","spark.executor.cores":"4","spark.history.ui.port":"49670","spark.ssl.historyServer.trustStorePassword":"*********(redacted)","spark.ssl.historyServer.protocol":"TLSv1.2","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"master1.company.com,master2.company.com","spark.history.kerberos.enabled":"true","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"https://master1.company.com:50030/proxy/application_1536831636016_59384,https://master2.company.com:50030/proxy/application_1536831636016_59384","spark.ssl.historyServer.trustStoreType":"JKS","spark.app.id":"application_1536831636016_59384"},"System Properties":{"java.io.tmpdir":"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/home/lajin","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/apache/releases/jdk1.8.0_121/jre/lib/amd64","user.dir":"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001","java.library.path":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native::/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.121-b13","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/apache/releases/jdk1.8.0_121/jre/lib/endorsed","java.runtime.version":"1.8.0_121-b13","java.vm.info":"mixed mode","java.ext.dirs":"/apache/releases/jdk1.8.0_121/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/apache/releases/jdk1.8.0_121/jre/lib/resources.jar:/apache/releases/jdk1.8.0_121/jre/lib/rt.jar:/apache/releases/jdk1.8.0_121/jre/lib/sunrsasign.jar:/apache/releases/jdk1.8.0_121/jre/lib/jsse.jar:/apache/releases/jdk1.8.0_121/jre/lib/jce.jar:/apache/releases/jdk1.8.0_121/jre/lib/charsets.jar:/apache/releases/jdk1.8.0_121/jre/lib/jfr.jar:/apache/releases/jdk1.8.0_121/jre/classes","file.encoding":"UTF-8","user.timezone":"Etc/GMT+7","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-514.21.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"lajin","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.SparkPi --jar file:/filer/home/lajin/SPARK-25865_new/spark-3.0.0-SNAPSHOT-bin-SPARK-25865_new/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --arg 100 --properties-file /hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__/__spark_conf__.properties","java.home":"/apache/releases/jdk1.8.0_121/jre","java.version":"1.8.0_121","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/azure-storage-2.2.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/xz-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-core-1.5.3-nohive.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-httpclient-3.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/gson-2.2.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-json-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-applications-unmanaged-am-launcher-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/objenesis-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/activation-1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/xz-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/paranamer-2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-io-2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/activation-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-guice-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-sharedcachemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-web-proxy-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/slf4j-api-1.7.10.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-timeline-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/junit-4.11.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/httpcore-4.2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guice-servlet-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/snappy-java-1.0.4.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-applicationhistoryservice-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-framework-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-cli-1.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/aopalliance-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-resourcemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/javax.inject-1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jettison-1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/hadoop-annotations-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/asm-3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-daemon-1.0.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-auth-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-nodemanager-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-io-2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/java-xmlbuilder-0.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/stax-api-1.0-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-codec-1.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jettison-1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-server-1.9.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-framework-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-compress-1.4.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/netty-all-4.0.23.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/xmlenc-0.52.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-applications-distributedshell-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-digester-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-client-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/httpclient-4.2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jetty-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-sharedcachemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-beanutils-1.7.0.jar":"System Classpath","/apache/confs/hive/conf":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/java-xmlbuilder-0.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsp-api-2.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guice-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsch-0.1.42.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/httpcore-4.2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-net-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-mapreduce-1.5.3-nohive.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-applicationhistoryservice-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4-tests.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-resourcemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guice-3.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-cli-1.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-annotations-2.2.3.jar":"System Classpath","/apache/confs/hadoop/conf":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/gson-2.2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/junit-4.11.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-nodemanager-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-nfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/httpcore-4.2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-client-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/aws-java-sdk-1.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-applications-distributedshell-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/xz-1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/gson-2.2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-json-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsr305-3.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jsr305-3.0.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-api-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-registry-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-registry-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-common-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/asm-3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-web-proxy-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-digester-1.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/servlet-api-2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xercesImpl-2.9.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-shuffle-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-io-2.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-digester-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-azure-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-recipes-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/avro-1.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-applications-unmanaged-am-launcher-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-tests-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/activation-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-timeline-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/xz-1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/guava-11.0.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/objenesis-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-shims-1.5.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/spark-2.2.1-yarn-shuffle.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/spark-2.2.1-yarn-shuffle.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jersey-core-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/okio-1.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsch-0.1.42.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-codec-1.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-lang-2.6.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/httpclient-4.2.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jettison-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-guice-1.9.jar":"System Classpath","/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-xc-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-api-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsch-0.1.42.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/fst-2.24.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-guice-1.9.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-app-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/guice-servlet-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jaxb-api-2.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-nfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-xc-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/guice-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-logging-1.1.3.jar":"System Classpath","/contrib/capacity-scheduler/*.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-net-3.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/javassist-3.18.1-GA.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-beanutils-1.7.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/okhttp-2.4.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-codec-1.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jetty-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/fst-2.24.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-tests-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jaxb-api-2.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/hamcrest-core-1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/mysql-connector-java-5.1.40-bin.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-core-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-common-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-aws-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-compress-1.4.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/httpclient-4.2.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guice-servlet-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/stax-api-1.0-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-annotations-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-lang-2.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-net-3.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-core-2.7.1.2.4.2.66-4.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark Pi","App ID":"application_1536831636016_59384","Timestamp":1546922023607,"User":"lajin","App Attempt ID":"1","Driver Logs":{"stdout":"https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stdout?start=-4096","stderr":"https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stderr?start=-4096"}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1546922034270,"Executor ID":"2","Executor Info":{"Host":"028.company.com","Total Cores":4,"Log Urls":{"stdout":"https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stdout?start=-4096","stderr":"https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"028.company.com","Port":46325},"Maximum Memory":97832140,"Timestamp":1546922034365,"Maximum Onheap Memory":97832140,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1546922035929,"Executor ID":"1","Executor Info":{"Host":"036.company.com","Total Cores":4,"Log Urls":{"stdout":"https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stdout?start=-4096","stderr":"https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stderr?start=-4096"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"036.company.com","Port":35126},"Maximum Memory":97832140,"Timestamp":1546922036120,"Maximum Onheap Memory":97832140,"Maximum Offheap Memory":0} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1546922036359,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"2\",\"name\":\"reduce\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Submission Time":1546922036383,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"2\",\"name\":\"reduce\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1546922036494,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1546922036517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1546922037414,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1546922037416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037421,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":86,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":834,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":114738938,"Value":114738938,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":134,"Value":134,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":296991078,"Value":296991078,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":725,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":296991078,"Executor Run Time":134,"Executor CPU Time":114738938,"Result Size":834,"JVM GC Time":86,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1546922036517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037424,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":86,"Value":172,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":1668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":116821463,"Value":231560401,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":134,"Value":268,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":250458033,"Value":547449111,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":1450,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":250458033,"Executor Run Time":134,"Executor CPU Time":116821463,"Result Size":834,"JVM GC Time":86,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1546922037431,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037432,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":93,"Value":265,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":2502,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":139987945,"Value":371548346,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":165,"Value":433,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":443768774,"Value":991217885,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":2175,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":443768774,"Executor Run Time":165,"Executor CPU Time":139987945,"Result Size":834,"JVM GC Time":93,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1546922037432,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037433,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":93,"Value":358,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":3336,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":143148800,"Value":514697146,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":165,"Value":598,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":257929339,"Value":1249147224,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":2900,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":257929339,"Executor Run Time":165,"Executor CPU Time":143148800,"Result Size":834,"JVM GC Time":93,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1546922037499,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1546922037414,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037500,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":7,"Value":365,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":4170,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":68955072,"Value":583652218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":69,"Value":667,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2463324,"Value":1251610548,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":2910,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":2463324,"Executor Run Time":69,"Executor CPU Time":68955072,"Result Size":834,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1546922037500,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1546922037416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037501,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":7,"Value":372,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":5004,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":69409062,"Value":653061280,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":77,"Value":744,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2197208,"Value":1253807756,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2912,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2197208,"Executor Run Time":77,"Executor CPU Time":69409062,"Result Size":834,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1546922037546,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1546922037499,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037547,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":5795,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":41068546,"Value":694129826,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":41,"Value":785,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1546605,"Value":1255354361,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2914,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1546605,"Executor Run Time":41,"Executor CPU Time":41068546,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1546922037583,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037584,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":280,"Value":652,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":6629,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":121468700,"Value":815598526,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":136,"Value":921,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":239941709,"Value":1495296070,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":3814,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":239941709,"Executor Run Time":136,"Executor CPU Time":121468700,"Result Size":834,"JVM GC Time":280,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1546922037587,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1546922037546,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037588,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":7420,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34831594,"Value":850430120,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":956,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2083011,"Value":1497379081,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":3816,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2083011,"Executor Run Time":35,"Executor CPU Time":34831594,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1546922037624,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1546922037587,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037624,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":8211,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31573348,"Value":882003468,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":988,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1326191,"Value":1498705272,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":3817,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1326191,"Executor Run Time":32,"Executor CPU Time":31573348,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1546922037633,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1546922036494,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037634,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":938,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":9045,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":174111771,"Value":1056115239,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":194,"Value":1182,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":414632138,"Value":1913337410,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":4717,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":414632138,"Executor Run Time":194,"Executor CPU Time":174111771,"Result Size":834,"JVM GC Time":286,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1546922037639,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1546922037583,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037639,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":944,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":9879,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":40168837,"Value":1096284076,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":46,"Value":1228,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2240849,"Value":1915578259,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":4720,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2240849,"Executor Run Time":46,"Executor CPU Time":40168837,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1546922037641,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037641,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":1230,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":10713,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":189780021,"Value":1286064097,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":203,"Value":1431,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":265791948,"Value":2181370207,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":5620,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":265791948,"Executor Run Time":203,"Executor CPU Time":189780021,"Result Size":834,"JVM GC Time":286,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1546922037642,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037642,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":1516,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":11547,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":184630125,"Value":1470694222,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":203,"Value":1634,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":241252253,"Value":2422622460,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":6520,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":241252253,"Executor Run Time":203,"Executor CPU Time":184630125,"Result Size":834,"JVM GC Time":286,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1546922037662,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1546922037624,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037662,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":12338,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31605508,"Value":1502299730,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":1665,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1157338,"Value":2423779798,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6522,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1157338,"Executor Run Time":31,"Executor CPU Time":31605508,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1546922037665,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1546922037431,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037665,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":13129,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":227941261,"Value":1730240991,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":228,"Value":1893,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1632598,"Value":2425412396,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6524,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1632598,"Executor Run Time":228,"Executor CPU Time":227941261,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1546922037671,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1546922037432,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037672,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":13920,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":234080439,"Value":1964321430,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":234,"Value":2127,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1842114,"Value":2427254510,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6526,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1842114,"Executor Run Time":234,"Executor CPU Time":234080439,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1546922037685,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1546922037642,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037685,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":14711,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36307061,"Value":2000628491,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":2164,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2588273,"Value":2429842783,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6528,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2588273,"Executor Run Time":37,"Executor CPU Time":36307061,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1546922037699,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1546922037662,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037700,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":15502,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32277689,"Value":2032906180,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":2196,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1148300,"Value":2430991083,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6530,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1148300,"Executor Run Time":32,"Executor CPU Time":32277689,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1546922037724,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1546922037685,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037724,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":16293,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32762038,"Value":2065668218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":2229,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1655404,"Value":2432646487,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6532,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1655404,"Executor Run Time":33,"Executor CPU Time":32762038,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1546922037740,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1546922037699,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037740,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":17084,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30991307,"Value":2096659525,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":2260,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2219464,"Value":2434865951,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6534,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2219464,"Executor Run Time":31,"Executor CPU Time":30991307,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1546922037744,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1546922037633,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037745,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":17875,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":104659378,"Value":2201318903,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":104,"Value":2364,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1617759,"Value":2436483710,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6537,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":1617759,"Executor Run Time":104,"Executor CPU Time":104659378,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1546922037758,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1546922037724,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037759,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":18666,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":29044222,"Value":2230363125,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2017183,"Value":2438500893,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6539,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2017183,"Executor Run Time":30,"Executor CPU Time":29044222,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1546922037779,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1546922037740,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037780,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":19457,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34945007,"Value":2265308132,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":2429,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":998897,"Value":2439499790,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6540,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":998897,"Executor Run Time":35,"Executor CPU Time":34945007,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1546922037781,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1546922037500,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037781,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":20248,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":275257849,"Value":2540565981,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":275,"Value":2704,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1522428,"Value":2441022218,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6542,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1522428,"Executor Run Time":275,"Executor CPU Time":275257849,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1546922037791,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1546922037758,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037792,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":20996,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28454753,"Value":2569020734,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":2734,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1143042,"Value":2442165260,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1143042,"Executor Run Time":30,"Executor CPU Time":28454753,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1546922037822,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1546922037781,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037822,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":21830,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36300767,"Value":2605321501,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":2769,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2138947,"Value":2444304207,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6545,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2138947,"Executor Run Time":35,"Executor CPU Time":36300767,"Result Size":834,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1546922037825,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1546922037791,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037825,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":22621,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28452489,"Value":2633773990,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":28,"Value":2797,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1182965,"Value":2445487172,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6547,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1182965,"Executor Run Time":28,"Executor CPU Time":28452489,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1546922037857,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1546922037825,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037858,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":23412,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28114352,"Value":2661888342,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":2826,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1597988,"Value":2447085160,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6548,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1597988,"Executor Run Time":29,"Executor CPU Time":28114352,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1546922037865,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1546922037822,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037866,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":24203,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":39843327,"Value":2701731669,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":39,"Value":2865,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":979795,"Value":2448064955,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6550,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":979795,"Executor Run Time":39,"Executor CPU Time":39843327,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1546922037890,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1546922037857,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037890,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":24994,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":27673779,"Value":2729405448,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":27,"Value":2892,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1462274,"Value":2449527229,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6552,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1462274,"Executor Run Time":27,"Executor CPU Time":27673779,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1546922037894,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1546922037639,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037894,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":25785,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":250723087,"Value":2980128535,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":250,"Value":3142,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1520638,"Value":2451047867,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6554,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1520638,"Executor Run Time":250,"Executor CPU Time":250723087,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1546922037917,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1546922037865,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037917,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1522,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":26619,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":40203057,"Value":3020331592,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":46,"Value":3188,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":987313,"Value":2452035180,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6556,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":987313,"Executor Run Time":46,"Executor CPU Time":40203057,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1546922037919,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1546922037665,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037919,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1528,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":27453,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":241618343,"Value":3261949935,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":248,"Value":3436,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1531323,"Value":2453566503,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6558,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1531323,"Executor Run Time":248,"Executor CPU Time":241618343,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1546922037922,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1546922037890,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037923,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":28201,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28219664,"Value":3290169599,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":3465,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1047569,"Value":2454614072,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1047569,"Executor Run Time":29,"Executor CPU Time":28219664,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1546922037949,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1546922037671,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037952,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1534,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":29035,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":239368713,"Value":3529538312,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":246,"Value":3711,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1606167,"Value":2456220239,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6561,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":1606167,"Executor Run Time":246,"Executor CPU Time":239368713,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1546922037952,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1546922037641,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037953,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":29826,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":281427189,"Value":3810965501,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":282,"Value":3993,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1654852,"Value":2457875091,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6563,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1654852,"Executor Run Time":282,"Executor CPU Time":281427189,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1546922037953,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1546922037744,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037953,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":30617,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":188231506,"Value":3999197007,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":188,"Value":4181,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1169748,"Value":2459044839,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6565,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1169748,"Executor Run Time":188,"Executor CPU Time":188231506,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1546922037954,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1546922037894,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037954,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":31408,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":42202483,"Value":4041399490,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":42,"Value":4223,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1131763,"Value":2460176602,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6567,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1131763,"Executor Run Time":42,"Executor CPU Time":42202483,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1546922037955,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1546922037919,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037955,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":32242,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32046930,"Value":4073446420,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4254,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1193077,"Value":2461369679,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6569,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1193077,"Executor Run Time":31,"Executor CPU Time":32046930,"Result Size":834,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1546922037990,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1546922037955,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037990,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":33033,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31475569,"Value":4104921989,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4285,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1060814,"Value":2462430493,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6571,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1060814,"Executor Run Time":31,"Executor CPU Time":31475569,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1546922037991,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1546922037954,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037992,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":33781,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33893704,"Value":4138815693,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":4320,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":995078,"Value":2463425571,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":995078,"Executor Run Time":35,"Executor CPU Time":33893704,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1546922038023,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1546922037991,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038024,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":34529,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":27659093,"Value":4166474786,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":4349,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":936644,"Value":2464362215,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":936644,"Executor Run Time":29,"Executor CPU Time":27659093,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1546922038025,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1546922037990,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038025,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":35320,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30657164,"Value":4197131950,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":4379,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1037555,"Value":2465399770,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6572,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1037555,"Executor Run Time":30,"Executor CPU Time":30657164,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1546922038060,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1546922038025,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038061,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":36111,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31510921,"Value":4228642871,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":4411,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":895198,"Value":2466294968,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6573,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":895198,"Executor Run Time":32,"Executor CPU Time":31510921,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1546922038063,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1546922038023,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038064,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1540,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":877,"Value":36988,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30458180,"Value":4259101051,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":4446,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":955927,"Value":2467250895,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6575,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":955927,"Executor Run Time":35,"Executor CPU Time":30458180,"Result Size":877,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1546922038095,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1546922038060,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038095,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":37779,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31390887,"Value":4290491938,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4477,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":893866,"Value":2468144761,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6576,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":893866,"Executor Run Time":31,"Executor CPU Time":31390887,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1546922038097,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1546922038063,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038097,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":38570,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28537105,"Value":4319029043,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":28,"Value":4505,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1137737,"Value":2469282498,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6578,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1137737,"Executor Run Time":28,"Executor CPU Time":28537105,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1546922038113,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1546922037779,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038113,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1546,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":39404,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":320279036,"Value":4639308079,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":327,"Value":4832,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2309726,"Value":2471592224,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6580,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2309726,"Executor Run Time":327,"Executor CPU Time":320279036,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1546922038131,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1546922038095,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038131,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":40195,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32175125,"Value":4671483204,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":4864,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":861437,"Value":2472453661,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6581,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":861437,"Executor Run Time":32,"Executor CPU Time":32175125,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1546922038137,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1546922037952,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038137,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1552,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":41029,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":175258771,"Value":4846741975,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":180,"Value":5044,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1272588,"Value":2473726249,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6583,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1272588,"Executor Run Time":180,"Executor CPU Time":175258771,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1546922038148,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1546922038097,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038148,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":41820,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":46067636,"Value":4892809611,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":47,"Value":5091,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1539643,"Value":2475265892,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6584,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1539643,"Executor Run Time":47,"Executor CPU Time":46067636,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1546922038168,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1546922038131,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038169,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":42611,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33746415,"Value":4926556026,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":5125,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":939846,"Value":2476205738,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6585,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":939846,"Executor Run Time":34,"Executor CPU Time":33746415,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1546922038175,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1546922037917,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038175,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":43402,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":253536114,"Value":5180092140,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":254,"Value":5379,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1121119,"Value":2477326857,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6586,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1121119,"Executor Run Time":254,"Executor CPU Time":253536114,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1546922038185,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1546922038148,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038185,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":44193,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33229424,"Value":5213321564,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":5412,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":856494,"Value":2478183351,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":856494,"Executor Run Time":33,"Executor CPU Time":33229424,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1546922038204,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1546922038168,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038204,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":44941,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31762584,"Value":5245084148,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":5444,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":819419,"Value":2479002770,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":819419,"Executor Run Time":32,"Executor CPU Time":31762584,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1546922038217,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1546922037922,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038217,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1558,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":45775,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":254396405,"Value":5499480553,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":260,"Value":5704,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1523395,"Value":2480526165,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6589,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1523395,"Executor Run Time":260,"Executor CPU Time":254396405,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1546922038218,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1546922038185,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038218,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":46523,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":29041950,"Value":5528522503,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":5734,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":920529,"Value":2481446694,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":920529,"Executor Run Time":30,"Executor CPU Time":29041950,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1546922038220,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1546922037953,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038221,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1564,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":47357,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":257839847,"Value":5786362350,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":264,"Value":5998,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1031380,"Value":2482478074,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6590,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1031380,"Executor Run Time":264,"Executor CPU Time":257839847,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1546922038239,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1546922038204,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038239,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":48191,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31072580,"Value":5817434930,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":6028,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":960275,"Value":2483438349,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6592,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":960275,"Executor Run Time":30,"Executor CPU Time":31072580,"Result Size":834,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1546922038251,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1546922038217,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038252,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":48982,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30559817,"Value":5847994747,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":6059,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":860099,"Value":2484298448,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":860099,"Executor Run Time":31,"Executor CPU Time":30559817,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1546922038253,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1546922037949,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038253,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":49773,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":296727303,"Value":6144722050,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":297,"Value":6356,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1526695,"Value":2485825143,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6593,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1526695,"Executor Run Time":297,"Executor CPU Time":296727303,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1546922038275,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1546922038239,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038276,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":50521,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33148590,"Value":6177870640,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6390,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":841498,"Value":2486666641,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":841498,"Executor Run Time":34,"Executor CPU Time":33148590,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1546922038291,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1546922038251,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038292,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":51269,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":35277840,"Value":6213148480,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":6426,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":981161,"Value":2487647802,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":981161,"Executor Run Time":36,"Executor CPU Time":35277840,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1546922038294,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1546922038137,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038295,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":52060,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":152692713,"Value":6365841193,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":153,"Value":6579,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1007896,"Value":2488655698,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6594,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1007896,"Executor Run Time":153,"Executor CPU Time":152692713,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1546922038313,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1546922038275,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038313,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":52808,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33306826,"Value":6399148019,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6613,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":956832,"Value":2489612530,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":956832,"Executor Run Time":34,"Executor CPU Time":33306826,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1546922038329,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1546922038291,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038330,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":53599,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33756595,"Value":6432904614,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":6646,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":882246,"Value":2490494776,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6595,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":882246,"Executor Run Time":33,"Executor CPU Time":33756595,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1546922038347,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1546922038313,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038348,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":54390,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30684763,"Value":6463589377,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":6676,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":943134,"Value":2491437910,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6597,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":943134,"Executor Run Time":30,"Executor CPU Time":30684763,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1546922038368,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1546922038329,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038368,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":55181,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34465927,"Value":6498055304,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6710,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":964913,"Value":2492402823,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6598,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":964913,"Executor Run Time":34,"Executor CPU Time":34465927,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1546922038390,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1546922038347,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038391,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1570,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":56015,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31793636,"Value":6529848940,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":6747,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":975320,"Value":2493378143,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6600,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":975320,"Executor Run Time":37,"Executor CPU Time":31793636,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1546922038409,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1546922038368,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038409,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":56806,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36274008,"Value":6566122948,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":6783,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":972306,"Value":2494350449,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6602,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":972306,"Executor Run Time":36,"Executor CPU Time":36274008,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1546922038416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1546922038175,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038416,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1576,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":57640,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":230475162,"Value":6796598110,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":236,"Value":7019,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":989658,"Value":2495340107,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6603,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":989658,"Executor Run Time":236,"Executor CPU Time":230475162,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1546922038424,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1546922038113,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038424,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1582,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":58474,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":301424170,"Value":7098022280,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":308,"Value":7327,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1012899,"Value":2496353006,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6604,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1012899,"Executor Run Time":308,"Executor CPU Time":301424170,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1546922038436,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1546922038253,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038436,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1588,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":59308,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":172594019,"Value":7270616299,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":179,"Value":7506,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1197294,"Value":2497550300,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6605,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1197294,"Executor Run Time":179,"Executor CPU Time":172594019,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1546922038448,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1546922038409,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038448,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":60099,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34408818,"Value":7305025117,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":7540,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":989551,"Value":2498539851,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6606,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":989551,"Executor Run Time":34,"Executor CPU Time":34408818,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1546922038483,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1546922038218,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038483,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1594,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":60890,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":254383686,"Value":7559408803,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":261,"Value":7801,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1135212,"Value":2499675063,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1135212,"Executor Run Time":261,"Executor CPU Time":254383686,"Result Size":791,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1546922038492,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1546922038448,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038493,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1600,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":61681,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34883743,"Value":7594292546,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":42,"Value":7843,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1012178,"Value":2500687241,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1012178,"Executor Run Time":42,"Executor CPU Time":34883743,"Result Size":791,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1546922038495,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1546922038220,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038495,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1606,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":62515,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":264938303,"Value":7859230849,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":270,"Value":8113,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":934827,"Value":2501622068,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6608,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":934827,"Executor Run Time":270,"Executor CPU Time":264938303,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1546922038507,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1546922038390,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038507,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":63263,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":112064631,"Value":7971295480,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":114,"Value":8227,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1053290,"Value":2502675358,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1053290,"Executor Run Time":114,"Executor CPU Time":112064631,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1546922038517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1546922038424,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038518,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":64054,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":89355066,"Value":8060650546,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":89,"Value":8316,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":998260,"Value":2503673618,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6610,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":998260,"Executor Run Time":89,"Executor CPU Time":89355066,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1546922038533,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1546922038492,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038533,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":64802,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":35725959,"Value":8096376505,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":8353,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1128802,"Value":2504802420,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1128802,"Executor Run Time":37,"Executor CPU Time":35725959,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1546922038536,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1546922038436,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038537,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":65593,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":96824193,"Value":8193200698,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":96,"Value":8449,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":857593,"Value":2505660013,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6612,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":857593,"Executor Run Time":96,"Executor CPU Time":96824193,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1546922038550,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1546922038294,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038550,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1612,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":66427,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":245593808,"Value":8438794506,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":251,"Value":8700,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1199629,"Value":2506859642,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6614,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1199629,"Executor Run Time":251,"Executor CPU Time":245593808,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1546922038551,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1546922038416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038552,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":67175,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":130530302,"Value":8569324808,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":132,"Value":8832,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1409511,"Value":2508269153,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1409511,"Executor Run Time":132,"Executor CPU Time":130530302,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1546922038574,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1546922038533,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038574,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":67966,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36971815,"Value":8606296623,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":8868,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1067084,"Value":2509336237,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6616,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1067084,"Executor Run Time":36,"Executor CPU Time":36971815,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1546922038612,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1546922038507,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038612,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":68757,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":100349075,"Value":8706645698,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":100,"Value":8968,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1073198,"Value":2510409435,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6617,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1073198,"Executor Run Time":100,"Executor CPU Time":100349075,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1546922038614,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1546922038495,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038614,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":69548,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":113541356,"Value":8820187054,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":114,"Value":9082,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":904680,"Value":2511314115,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":904680,"Executor Run Time":114,"Executor CPU Time":113541356,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1546922038621,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1546922038517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038621,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":70339,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":100446517,"Value":8920633571,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":100,"Value":9182,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":895108,"Value":2512209223,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6618,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":895108,"Executor Run Time":100,"Executor CPU Time":100446517,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1546922038636,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1546922038536,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038637,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":71130,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":96681743,"Value":9017315314,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":97,"Value":9279,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":881317,"Value":2513090540,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":881317,"Executor Run Time":97,"Executor CPU Time":96681743,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1546922038640,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1546922038574,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038641,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":71921,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":61426798,"Value":9078742112,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":61,"Value":9340,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1009168,"Value":2514099708,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6620,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1009168,"Executor Run Time":61,"Executor CPU Time":61426798,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1546922038664,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1546922038551,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038664,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":72669,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":108784588,"Value":9187526700,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":110,"Value":9450,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1001879,"Value":2515101587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1001879,"Executor Run Time":110,"Executor CPU Time":108784588,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1546922038680,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1546922038614,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038681,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":73460,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":60115853,"Value":9247642553,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":61,"Value":9511,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1466213,"Value":2516567800,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6621,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1466213,"Executor Run Time":61,"Executor CPU Time":60115853,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1546922038483,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038705,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":74251,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":214285427,"Value":9461927980,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":214,"Value":9725,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1060594,"Value":2517628394,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6623,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1060594,"Executor Run Time":214,"Executor CPU Time":214285427,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1546922038640,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038705,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":75042,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":57994238,"Value":9519922218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":58,"Value":9783,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1405974,"Value":2519034368,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6625,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1405974,"Executor Run Time":58,"Executor CPU Time":57994238,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1546922038612,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038717,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":75833,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":101684514,"Value":9621606732,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":101,"Value":9884,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":968858,"Value":2520003226,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6627,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":968858,"Executor Run Time":101,"Executor CPU Time":101684514,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1546922038680,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038724,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":76624,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":38985492,"Value":9660592224,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":39,"Value":9923,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1411378,"Value":2521414604,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6629,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1411378,"Executor Run Time":39,"Executor CPU Time":38985492,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1546922038550,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038725,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":77415,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":169783721,"Value":9830375945,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":170,"Value":10093,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":805689,"Value":2522220293,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6630,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":805689,"Executor Run Time":170,"Executor CPU Time":169783721,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1546922038621,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038726,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":78206,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":101866107,"Value":9932242052,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":102,"Value":10195,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":805277,"Value":2523025570,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6631,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":805277,"Executor Run Time":102,"Executor CPU Time":101866107,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1546922038636,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038729,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":78954,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":89123011,"Value":10021365063,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":90,"Value":10285,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":896962,"Value":2523922532,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":896962,"Executor Run Time":90,"Executor CPU Time":89123011,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1546922038664,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038733,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":79745,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":64532656,"Value":10085897719,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":64,"Value":10349,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1115835,"Value":2525038367,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6633,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1115835,"Executor Run Time":64,"Executor CPU Time":64532656,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":211171816,"JVMOffHeapMemory":90237256,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":4876,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":4876,"OffHeapUnifiedMemory":0,"DirectPoolMemory":806275,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2646888448,"ProcessTreeJVMRSSMemory":520900608,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":8,"MinorGCTime":374,"MajorGCCount":3,"MajorGCTime":170}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Submission Time":1546922036383,"Completion Time":1546922038734,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":10349,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":1612,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":79745,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2525038367,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":10085897719,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":6633,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1546922038738,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1546922038745} diff --git a/core/src/test/resources/spark-events/application_1538416563558_0014 b/core/src/test/resources/spark-events/application_1538416563558_0014 deleted file mode 100644 index 000288dbc454..000000000000 --- a/core/src/test/resources/spark-events/application_1538416563558_0014 +++ /dev/null @@ -1,190 +0,0 @@ -{"Event":"SparkListenerLogStart","Spark Version":"2.5.0-SNAPSHOT"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"rezamemory-1.gce.something.com","Port":43959},"Maximum Memory":384093388,"Timestamp":1538440967690,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_121/jre","Java Version":"1.8.0_121 (Oracle Corporation)","Scala Version":"version 2.11.12"},"Spark Properties":{"spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.yarn.jars":"local:/opt/some/path/lib/spark2/jars/*","spark.driver.host":"rezamemory-1.gce.something.com","spark.serializer.objectStreamReset":"100","spark.eventLog.enabled":"true","spark.executor.heartbeatInterval":"100ms","spark.hadoop.mapreduce.application.classpath":"","spark.driver.port":"35918","spark.shuffle.service.enabled":"true","spark.rdd.compress":"True","spark.driver.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.executorEnv.PYTHONPATH":"/opt/some/path/lib/spark2/python/lib/py4j-0.10.7-src.zip/opt/some/path/lib/spark2/python/lib/pyspark.zip","spark.yarn.historyServer.address":"http://rezamemory-1.gce.something.com:18089","spark.app.name":"PythonBisectingKMeansExample","spark.ui.killEnabled":"true","spark.sql.hive.metastore.jars":"${env:HADOOP_COMMON_HOME}/../hive/lib/*:${env:HADOOP_COMMON_HOME}/client/*","spark.dynamicAllocation.schedulerBacklogTimeout":"1","spark.yarn.am.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.scheduler.mode":"FIFO","spark.eventLog.logStageExecutorMetrics.enabled":"true","spark.yarn.config.gatewayPath":"/opt/cloudera/parcels","spark.executor.id":"driver","spark.yarn.config.replacementPath":"{{HADOOP_COMMON_HOME}}/../../..","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.submit.deployMode":"client","spark.shuffle.service.port":"7337","spark.master":"yarn","spark.authenticate":"false","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.eventLog.dir":"hdfs://rezamemory-1.gce.something.com:8020/user/spark/spark2ApplicationHistory","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.hadoop.yarn.application.classpath":"","spark.driver.appUIAddress":"http://rezamemory-1.gce.something.com:4040","spark.yarn.isPython":"true","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"rezamemory-1.gce.something.com","spark.dynamicAllocation.minExecutors":"0","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://rezamemory-1.gce.something.com:8088/proxy/application_1538416563558_0014","spark.dynamicAllocation.executorIdleTimeout":"60","spark.app.id":"application_1538416563558_0014","spark.sql.hive.metastore.version":"1.1.0"},"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":"/root","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_121/jre/lib/amd64","user.dir":"/","java.library.path":":/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native:/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.121-b13","jetty.git.hash":"unknown","java.endorsed.dirs":"/usr/java/jdk1.8.0_121/jre/lib/endorsed","java.runtime.version":"1.8.0_121-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_121/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_121/jre/lib/resources.jar:/usr/java/jdk1.8.0_121/jre/lib/rt.jar:/usr/java/jdk1.8.0_121/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_121/jre/lib/jsse.jar:/usr/java/jdk1.8.0_121/jre/lib/jce.jar:/usr/java/jdk1.8.0_121/jre/lib/charsets.jar:/usr/java/jdk1.8.0_121/jre/lib/jfr.jar:/usr/java/jdk1.8.0_121/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-693.5.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"root","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.executor.heartbeatInterval=100ms --conf spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true --conf spark.eventLog.logStageExecutorMetrics.enabled=true ./opt/some/path/lib/spark2/examples/src/main/python/mllib/bisecting_k_means_example.py","java.home":"/usr/java/jdk1.8.0_121/jre","java.version":"1.8.0_121","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/opt/some/path/lib/spark2/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/netty-3.10.5.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/validation-api-1.1.0.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-annotations-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-azure-datalake-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jaxb-impl-2.2.3-1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jasper-compiler-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/logredactor-1.0.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-streaming_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-common-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guice-servlet-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-hadoop-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-hadoop-1.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-jackson-1.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-server-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jtransforms-2.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aircompressor-0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-el-1.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/minlog-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-daemon-1.0.13.jar":"System Classpath","/opt/some/path/lib/spark2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-crypto-1.0.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-mllib-local_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-openstack-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-lang3-3.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/univocity-parsers-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javassist-3.18.1-GA.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-api-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-registry-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/activation-1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/objenesis-2.5.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-xc-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-applications-unmanaged-am-launcher-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-repl_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hue-plugins-3.9.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-digester-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-resourcemanager-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-math3-3.1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/activation-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-tests-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/snappy-java-1.0.4.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-json-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-kvstore_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-common-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-guice-1.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-archives-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-archive-logs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jetty-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-examples-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/snappy-java-1.1.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-web-proxy-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-mapreduce-1.5.2-nohive.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsch-0.1.42.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-jvm-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.annotation-api-1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/pyrolite-4.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-jackson-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-client-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jline-2.11.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-scala_2.10-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/spark-streaming-kafka-0-8_2.11-2.2.0.cloudera1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsp-api-2.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-nodemanager-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-logging-1.1.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-compiler-3.0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-generator-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-format-2.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-server-1.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-common-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jetty-6.1.26.cloudera.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/mockito-all-1.8.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-core-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jsp-api-2.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-unsafe_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/oro-2.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-hs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-codec-1.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xmlenc-0.52.jar":"System Classpath","/opt/some/path/lib/spark2/jars/opencsv-2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xbean-asm6-shaded-4.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/javax.inject-1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-encoding-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-library-2.11.12.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/log4j-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/LICENSE.txt":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/netty-3.9.9.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-api-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/httpcore-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jettison-1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/zookeeper-3.4.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-core-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-auth-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-core-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-network-shuffle_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-beanutils-1.9.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/chill_2.11-0.9.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-core-2.6.7.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/paranamer-2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/janino-3.0.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jetty-util-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/py4j-0.10.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/ivy-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-lang-2.6.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-client-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/stream-2.7.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xml-apis-1.3.04.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/kafka_2.11-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/metrics-core-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-applicationhistoryservice-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/conf/":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guice-servlet-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/slf4j-api-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-configuration-1.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xz-1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-tools-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-format-0.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/okio-1.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/compress-lzf-1.0.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-jobclient-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hppc-0.7.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/stax-api-1.0-2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-yarn_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-applications-distributedshell-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/joda-time-2.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-sls-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jets3t-0.9.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guice-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-streaming-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guava-14.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hamcrest-core-1.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/aws-java-sdk-bundle-1.11.134.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-client-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-hs-plugins-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-gridmix-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xz-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-pig-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-guava-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-compiler-2.11.12.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-sql_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-app-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/java-xmlbuilder-0.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/slf4j-api-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-hadoop-bundle-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-shuffle-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-pig-bundle-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-digester-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-json-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-codec-1.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-beanutils-1.7.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-catalyst_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-common-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-parser-combinators_2.11-1.1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jetty-util-6.1.26.cloudera.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/httpclient-4.5.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/antlr4-runtime-4.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-lang-2.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-mllib_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-app-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/machinist_2.11-0.6.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-core_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spire_2.11-0.13.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-xc-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-thrift-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/opt/some/path/lib/spark2/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-annotations-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-io-2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-annotations-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-core-1.5.2-nohive.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-net-3.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-memory-0.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-graphx_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-core-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/okhttp-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0-sources.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-datajoin-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-aws-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-reflect-2.11.12.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-net-3.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-databind-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-column-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xmlenc-0.52.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/kafka-clients-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-io-2.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/lz4-java-1.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/core-1.1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-vector-0.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-azure-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0-javadoc.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-nfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-client-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/breeze_2.11-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/httpcore-4.4.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-core-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/servlet-api-2.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-math3-3.4.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.inject-1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-jaxrs-1.8.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/spark-1.6.0-cdh5.12.0-yarn-shuffle.jar":"System Classpath","/opt/some/path/lib/spark2/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/zookeeper-3.4.5-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/httpclient-4.2.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-graphite-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-compress-1.4.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-sketch_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-network-common_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/gson-2.2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-cascading-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-auth-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-shims-1.5.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/log4j-1.2.17.jar":"System Classpath","/opt/some/path/lib/spark2/jars/stax-api-1.0-2.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/zkclient-0.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/paranamer-2.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/some/path/lib/spark2/jars/gson-2.2.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-tags_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-configuration-1.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-hdfs-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guice-3.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jsr305-1.3.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-client-2.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/conf/yarn-conf/":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/azure-data-lake-store-sdk-2.1.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-distcp-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/junit-4.11.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-extras-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jasper-runtime-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-client-2.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-1.8.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-compress-1.8.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsr305-3.0.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guava-11.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/asm-3.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/avro-1.7.6-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-client-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-cli-1.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-scrooge_2.10-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-cli-1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-ant-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-launcher_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-nativetask-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-nfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-encoding-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-shuffle-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-avro-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/opt/some/path/lib/spark2/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-test-hadoop2-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-column-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/slf4j-log4j12-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-protobuf-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-ipc-1.8.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arpack_combined_all-0.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/netty-all-4.1.17.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/chill-java-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/NOTICE.txt":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-common-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-mapper-asl-1.8.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-annotations-2.6.7.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-rumen-2.6.0-cdh5.12.0.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"PythonBisectingKMeansExample","App ID":"application_1538416563558_0014","Timestamp":1538440959580,"User":"root"} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1538440969009,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"3\",\"name\":\"first\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440969044,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"3\",\"name\":\"first\"}"}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538440973727,"Executor ID":"1","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000002/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000002/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1538440973735,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Port":46411},"Maximum Memory":384093388,"Timestamp":1538440973890,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1538440973735,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440977628,"Failed":false,"Killed":false,"Accumulables":[{"ID":23,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.jvmGCTime","Update":208,"Value":208,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.resultSize","Update":1448,"Value":1448,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.executorCpuTime","Update":1105071149,"Value":1105071149,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorRunTime","Update":2307,"Value":2307,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorDeserializeCpuTime","Update":651096062,"Value":651096062,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeTime","Update":1322,"Value":1322,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1322,"Executor Deserialize CPU Time":651096062,"Executor Run Time":2307,"Executor CPU Time":1105071149,"Result Size":1448,"JVM GC Time":208,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":72,"Records Read":4},"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":256071440,"JVMOffHeapMemory":92211424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":333371,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":333371,"OffHeapUnifiedMemory":0,"DirectPoolMemory":134726,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4926242816,"ProcessTreeJVMRSSMemory":525656064,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":182536928,"JVMOffHeapMemory":58263224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1086483,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1086483,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20304,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3009855488,"ProcessTreeJVMRSSMemory":404488192,"ProcessTreePythonVMemory":626200576,"ProcessTreePythonRSSMemory":69218304,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440969044,"Completion Time":1538440977644,"Accumulables":[{"ID":23,"Name":"internal.metrics.input.recordsRead","Value":4,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorDeserializeCpuTime","Value":651096062,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.resultSize","Value":1448,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"internal.metrics.input.bytesRead","Value":72,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.executorCpuTime","Value":1105071149,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeTime","Value":1322,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorRunTime","Value":2307,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.jvmGCTime","Value":208,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1538440977650,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1538440977784,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[1,2],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440977793,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1538440977816,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":1,"Attempt":0,"Launch Time":1538440978659,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1538440977816,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440978683,"Failed":false,"Killed":false,"Accumulables":[{"ID":48,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Update":13535058,"Value":13535058,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":1088,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Update":202227536,"Value":202227536,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Update":705,"Value":705,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Update":65694833,"Value":65694833,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Update":119,"Value":119,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":119,"Executor Deserialize CPU Time":65694833,"Executor Run Time":705,"Executor CPU Time":202227536,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":13535058,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":8},"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":2,"Index":1,"Attempt":0,"Launch Time":1538440978659,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440978820,"Failed":false,"Killed":false,"Accumulables":[{"ID":48,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Update":289555,"Value":13824613,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2176,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Update":36560031,"Value":238787567,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Update":120,"Value":825,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7042587,"Value":72737420,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":127,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7042587,"Executor Run Time":120,"Executor CPU Time":36560031,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":289555,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"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":292935952,"JVMOffHeapMemory":95141200,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":351534,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":351534,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4929392640,"ProcessTreeJVMRSSMemory":539996160,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":215586960,"JVMOffHeapMemory":60718904,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1492038,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1492038,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20637,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3014057984,"ProcessTreeJVMRSSMemory":422723584,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440977793,"Completion Time":1538440978821,"Accumulables":[{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Value":127,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Value":2176,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":356,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Value":238787567,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Value":13824613,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Value":825,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Value":72737420,"Internal":true,"Count Failed Values":true},{"ID":48,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440978830,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Launch Time":1538440978844,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1538440979033,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Launch Time":1538440978844,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979050,"Failed":false,"Killed":false,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":356,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":992,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Update":88389028,"Value":88389028,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Update":122,"Value":122,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Update":27126551,"Value":27126551,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Update":45,"Value":45,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":45,"Executor Deserialize CPU Time":27126551,"Executor Run Time":122,"Executor CPU Time":88389028,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":356,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1538440979033,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979084,"Failed":false,"Killed":false,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Update":0,"Value":992,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Update":1706,"Value":3534,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Update":15055355,"Value":103444383,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Update":26,"Value":148,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4722422,"Value":31848973,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":50,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4722422,"Executor Run Time":26,"Executor CPU Time":15055355,"Result Size":1706,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":303792496,"JVMOffHeapMemory":95545824,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":371127,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":371127,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4931497984,"ProcessTreeJVMRSSMemory":549777408,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":227393200,"JVMOffHeapMemory":61799392,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":463135,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":463135,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20637,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016163328,"ProcessTreeJVMRSSMemory":436539392,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440978830,"Completion Time":1538440979086,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Value":148,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Value":3534,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Value":31848973,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Value":992,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Value":103444383,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":2,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":356,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Value":50,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1538440979087,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1538440979161,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[3,4],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979163,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1538440979184,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":1,"Attempt":0,"Launch Time":1538440979344,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1538440979184,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979348,"Failed":false,"Killed":false,"Accumulables":[{"ID":98,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Update":259310,"Value":259310,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Update":40081727,"Value":40081727,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Update":98,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Update":24271689,"Value":24271689,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Update":39,"Value":39,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":39,"Executor Deserialize CPU Time":24271689,"Executor Run Time":98,"Executor CPU Time":40081727,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":259310,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":1,"Attempt":0,"Launch Time":1538440979344,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979441,"Failed":false,"Killed":false,"Accumulables":[{"ID":98,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Update":221381,"Value":480691,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Update":23089017,"Value":63170744,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Update":74,"Value":172,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3471167,"Value":27742856,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":43,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3471167,"Executor Run Time":74,"Executor CPU Time":23089017,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":221381,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":3,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":318926040,"JVMOffHeapMemory":96521592,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":391718,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":391718,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4932550656,"ProcessTreeJVMRSSMemory":569753600,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":3,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":236711480,"JVMOffHeapMemory":62683008,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":483726,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":483726,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20922,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3019313152,"ProcessTreeJVMRSSMemory":445640704,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979163,"Completion Time":1538440979444,"Accumulables":[{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Value":27742856,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Value":43,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Value":63170744,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Value":480691,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Value":172,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979446,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1538440979462,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1538440979527,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1538440979462,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979528,"Failed":false,"Killed":false,"Accumulables":[{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Update":17714408,"Value":17714408,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Update":30,"Value":30,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12579502,"Value":12579502,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":22,"Executor Deserialize CPU Time":12579502,"Executor Run Time":30,"Executor CPU Time":17714408,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1538440979527,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979572,"Failed":false,"Killed":false,"Accumulables":[{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Update":1828,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Update":16462125,"Value":34176533,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Update":16,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3190663,"Value":15770165,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":26,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3190663,"Executor Run Time":16,"Executor CPU Time":16462125,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":1,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":4,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":329919832,"JVMOffHeapMemory":96756344,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":413740,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":413740,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4935208960,"ProcessTreeJVMRSSMemory":585252864,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":4,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":242876648,"JVMOffHeapMemory":62975784,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":505748,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":505748,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20922,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3019313152,"ProcessTreeJVMRSSMemory":451244032,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979446,"Completion Time":1538440979573,"Accumulables":[{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Value":26,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Value":34176533,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Value":46,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Value":3656,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Value":15770165,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1538440979573,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1538440979609,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[5,6],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979619,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1538440979638,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":1,"Attempt":0,"Launch Time":1538440979754,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1538440979638,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979756,"Failed":false,"Killed":false,"Accumulables":[{"ID":148,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Update":272852,"Value":272852,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Update":23042622,"Value":23042622,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Update":76,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13112180,"Value":13112180,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":13112180,"Executor Run Time":76,"Executor CPU Time":23042622,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":272852,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":1,"Attempt":0,"Launch Time":1538440979754,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979851,"Failed":false,"Killed":false,"Accumulables":[{"ID":148,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Update":229882,"Value":502734,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Update":22093052,"Value":45135674,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Update":81,"Value":157,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3862579,"Value":16974759,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":32,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3862579,"Executor Run Time":81,"Executor CPU Time":22093052,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":229882,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":5,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":341682304,"JVMOffHeapMemory":97514672,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":434309,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":434309,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4935254016,"ProcessTreeJVMRSSMemory":597999616,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":5,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":252029672,"JVMOffHeapMemory":63463032,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":526317,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":526317,"OffHeapUnifiedMemory":0,"DirectPoolMemory":21041,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3020365824,"ProcessTreeJVMRSSMemory":458960896,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979619,"Completion Time":1538440979852,"Accumulables":[{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Value":502734,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Value":157,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Value":16974759,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":148,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Value":45135674,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Value":32,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979854,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1538440979869,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1538440979920,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1538440979869,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979921,"Failed":false,"Killed":false,"Accumulables":[{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Update":15546330,"Value":15546330,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Update":19,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11263754,"Value":11263754,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":22,"Executor Deserialize CPU Time":11263754,"Executor Run Time":19,"Executor CPU Time":15546330,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1538440979920,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979972,"Failed":false,"Killed":false,"Accumulables":[{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":157,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Update":1871,"Value":3699,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Update":15089701,"Value":30636031,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Update":27,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3045280,"Value":14309034,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":25,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3045280,"Executor Run Time":27,"Executor CPU Time":15089701,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":6,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":350990264,"JVMOffHeapMemory":97710440,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":456312,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":456312,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4932550656,"ProcessTreeJVMRSSMemory":604299264,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979854,"Completion Time":1538440979973,"Accumulables":[{"ID":155,"Name":"internal.metrics.resultSize","Value":3699,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Value":25,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Value":30636031,"Internal":true,"Count Failed Values":true},{"ID":157,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Value":46,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Value":14309034,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1538440979974,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1538440980008,"Stage Infos":[{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[7,8],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440980015,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1538440980049,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Port":46411},"Timestamp":1538440980522} -{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440980759,"Executor ID":"1","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000002 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000002\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} -{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"1","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000002 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000002\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1538440980049,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440980757,"Failed":true,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538440986317,"Executor ID":"2","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000003/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000003/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":0,"Attempt":1,"Launch Time":1538440986317,"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Port":39119},"Maximum Memory":384093388,"Timestamp":1538440986696,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538440988793,"Executor ID":"3","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000004/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000004/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":1,"Attempt":0,"Launch Time":1538440988793,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Port":40911},"Maximum Memory":384093388,"Timestamp":1538440989162,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Port":39119},"Timestamp":1538440993798} -{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"2","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000003 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000003\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":14,"Index":0,"Attempt":1,"Launch Time":1538440986317,"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440994010,"Failed":true,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440994012,"Executor ID":"2","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000003 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000003\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} -{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":2,"Launch Time":1538440995449,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":1,"Attempt":0,"Launch Time":1538440988793,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440995450,"Failed":false,"Killed":false,"Accumulables":[{"ID":198,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Update":10065137,"Value":10065137,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":1088,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.jvmGCTime","Update":360,"Value":360,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Update":1705,"Value":1705,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Update":1406669099,"Value":1406669099,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Update":4128,"Value":4128,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Update":726605764,"Value":726605764,"Internal":true,"Count Failed Values":true},{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Update":1995,"Value":1995,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1995,"Executor Deserialize CPU Time":726605764,"Executor Run Time":4128,"Executor CPU Time":1406669099,"Result Size":1705,"JVM GC Time":360,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":10065137,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":2,"Launch Time":1538440995449,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440995696,"Failed":false,"Killed":false,"Accumulables":[{"ID":198,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Update":293846,"Value":10358983,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Update":1662,"Value":3367,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Update":91844758,"Value":1498513857,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Update":220,"Value":4348,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8316162,"Value":734921926,"Internal":true,"Count Failed Values":true},{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":2004,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8316162,"Executor Run Time":220,"Executor CPU Time":91844758,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":293846,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":201931120,"JVMOffHeapMemory":58230320,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1094710,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1094710,"OffHeapUnifiedMemory":0,"DirectPoolMemory":45633,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3023769600,"ProcessTreeJVMRSSMemory":410324992,"ProcessTreePythonVMemory":285470720,"ProcessTreePythonRSSMemory":30171136,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":195471784,"JVMOffHeapMemory":100867584,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":476885,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":476885,"OffHeapUnifiedMemory":0,"DirectPoolMemory":171571,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4971368448,"ProcessTreeJVMRSSMemory":663375872,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":258718872,"JVMOffHeapMemory":63737056,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":548320,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":548320,"OffHeapUnifiedMemory":0,"DirectPoolMemory":21084,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3021418496,"ProcessTreeJVMRSSMemory":466001920,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":223684056,"JVMOffHeapMemory":60665000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1482102,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1482102,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20318,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3015626752,"ProcessTreeJVMRSSMemory":404672512,"ProcessTreePythonVMemory":958963712,"ProcessTreePythonRSSMemory":106639360,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440980015,"Completion Time":1538440995697,"Accumulables":[{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Value":2004,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Value":4348,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Value":10358983,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.jvmGCTime","Value":360,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Value":3367,"Internal":true,"Count Failed Values":true},{"ID":198,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Value":734921926,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Value":1498513857,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440995698,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1538440995710,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Port":40911},"Timestamp":1538440996257} -{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"3","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000004 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000004\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1538440995710,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440996467,"Failed":true,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440996468,"Executor ID":"3","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000004 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000004\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538441002826,"Executor ID":"4","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000005/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000005/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":0,"Attempt":1,"Launch Time":1538441002828,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538441003031,"Executor ID":"5","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000006/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000006/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":1,"Attempt":0,"Launch Time":1538441003032,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Port":39248},"Maximum Memory":384093388,"Timestamp":1538441003132,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Port":43165},"Maximum Memory":384093388,"Timestamp":1538441003383,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":0,"Attempt":1,"Launch Time":1538441002828,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441006147,"Failed":false,"Killed":false,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Update":350,"Value":350,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Update":1914,"Value":1914,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Update":219243972,"Value":219243972,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Update":893,"Value":893,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Update":717217987,"Value":717217987,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Update":1972,"Value":1972,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1972,"Executor Deserialize CPU Time":717217987,"Executor Run Time":893,"Executor CPU Time":219243972,"Result Size":1914,"JVM GC Time":350,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":1,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":178,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":1,"Attempt":0,"Launch Time":1538441003032,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441006584,"Failed":false,"Killed":false,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Update":10,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Update":270,"Value":620,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Update":1914,"Value":3828,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Update":210863492,"Value":430107464,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Update":412,"Value":1305,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Update":727356712,"Value":1444574699,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Update":2604,"Value":4576,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2604,"Executor Deserialize CPU Time":727356712,"Executor Run Time":412,"Executor CPU Time":210863492,"Result Size":1914,"JVM GC Time":270,"Result Serialization Time":10,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":2,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":355,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":263995432,"JVMOffHeapMemory":101978136,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":498888,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":498888,"OffHeapUnifiedMemory":0,"DirectPoolMemory":191656,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5008089088,"ProcessTreeJVMRSSMemory":663732224,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"5","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":150497592,"JVMOffHeapMemory":45958576,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":22003,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":22003,"OffHeapUnifiedMemory":0,"DirectPoolMemory":3446,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2984218624,"ProcessTreeJVMRSSMemory":325042176,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":181352744,"JVMOffHeapMemory":47061200,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":22003,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":22003,"OffHeapUnifiedMemory":0,"DirectPoolMemory":11272,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3013332992,"ProcessTreeJVMRSSMemory":416645120,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":226223752,"JVMOffHeapMemory":60840424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":433558,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":433558,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20318,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016937472,"ProcessTreeJVMRSSMemory":406044672,"ProcessTreePythonVMemory":958963712,"ProcessTreePythonRSSMemory":106639360,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440995698,"Completion Time":1538441006585,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1444574699,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Value":3828,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Value":12,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Value":4576,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Value":430107464,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Value":1305,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Value":620,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1538441006585,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1538441006610,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[9,10],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441006612,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":0,"Attempt":0,"Launch Time":1538441006622,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":1,"Attempt":0,"Launch Time":1538441006623,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Port":39248},"Timestamp":1538441010070} -{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Port":43165},"Timestamp":1538441010233} -{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"4","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000005 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000005\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":20,"Index":0,"Attempt":0,"Launch Time":1538441006622,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441010280,"Failed":true,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441010281,"Executor ID":"4","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000005 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000005\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} -{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"5","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000006 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000006\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":21,"Index":1,"Attempt":0,"Launch Time":1538441006623,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441010484,"Failed":true,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441010485,"Executor ID":"5","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000006 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000006\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538441015443,"Executor ID":"6","Executor Info":{"Host":"rezamemory-3.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000007/root/stdout?start=-4096","stderr":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000007/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":1,"Attempt":1,"Launch Time":1538441015444,"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Port":45593},"Maximum Memory":384093388,"Timestamp":1538441015852,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538441020314,"Executor ID":"7","Executor Info":{"Host":"rezamemory-3.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000008/root/stdout?start=-4096","stderr":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000008/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":0,"Attempt":1,"Launch Time":1538441020315,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Port":40992},"Maximum Memory":384093388,"Timestamp":1538441020602,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Port":45593},"Timestamp":1538441022942} -{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"6","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000007 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000007\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":22,"Index":1,"Attempt":1,"Launch Time":1538441015444,"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441023152,"Failed":true,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441023153,"Executor ID":"6","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000007 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000007\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} -{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":1,"Attempt":2,"Launch Time":1538441025899,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":0,"Attempt":1,"Launch Time":1538441020315,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441025900,"Failed":false,"Killed":false,"Accumulables":[{"ID":248,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Update":3971129,"Value":3971129,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":231,"Name":"internal.metrics.jvmGCTime","Update":244,"Value":244,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Update":1705,"Value":1705,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Update":1268816374,"Value":1268816374,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Update":2978,"Value":2978,"Internal":true,"Count Failed Values":true},{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Update":714859741,"Value":714859741,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Update":2106,"Value":2106,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2106,"Executor Deserialize CPU Time":714859741,"Executor Run Time":2978,"Executor CPU Time":1268816374,"Result Size":1705,"JVM GC Time":244,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":3971129,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":1,"Attempt":2,"Launch Time":1538441025899,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026136,"Failed":false,"Killed":false,"Accumulables":[{"ID":248,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Update":265841,"Value":4236970,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":232,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Update":1705,"Value":3410,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Update":88980290,"Value":1357796664,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Update":201,"Value":3179,"Internal":true,"Count Failed Values":true},{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8550572,"Value":723410313,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":2119,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":8550572,"Executor Run Time":201,"Executor CPU Time":88980290,"Result Size":1705,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":265841,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":332727504,"JVMOffHeapMemory":103237664,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":519462,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":519462,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228406,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5011247104,"ProcessTreeJVMRSSMemory":658915328,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"5","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":184519808,"JVMOffHeapMemory":58341088,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1116714,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1116714,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20420,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2998673408,"ProcessTreeJVMRSSMemory":378527744,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":218694008,"JVMOffHeapMemory":60757008,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1482103,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1482103,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20668,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3020120064,"ProcessTreeJVMRSSMemory":423698432,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":220189424,"JVMOffHeapMemory":59534504,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1116714,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1116714,"OffHeapUnifiedMemory":0,"DirectPoolMemory":27895,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3024392192,"ProcessTreeJVMRSSMemory":431939584,"ProcessTreePythonVMemory":283738112,"ProcessTreePythonRSSMemory":27226112,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"6","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":208356192,"JVMOffHeapMemory":58297728,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1094711,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1094711,"OffHeapUnifiedMemory":0,"DirectPoolMemory":27296,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3027820544,"ProcessTreeJVMRSSMemory":439750656,"ProcessTreePythonVMemory":286220288,"ProcessTreePythonRSSMemory":30846976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441006612,"Completion Time":1538441026137,"Accumulables":[{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Value":723410313,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Value":2119,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Value":1357796664,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true},{"ID":232,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Value":3179,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Value":4236970,"Internal":true,"Count Failed Values":true},{"ID":231,"Name":"internal.metrics.jvmGCTime","Value":244,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Value":3410,"Internal":true,"Count Failed Values":true},{"ID":248,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026138,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1538441026147,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1538441026309,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1538441026147,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026311,"Failed":false,"Killed":false,"Accumulables":[{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":254,"Name":"internal.metrics.executorCpuTime","Update":80311930,"Value":80311930,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Update":89,"Value":89,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29610969,"Value":29610969,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":62,"Executor Deserialize CPU Time":29610969,"Executor Run Time":89,"Executor CPU Time":80311930,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1538441026309,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026375,"Failed":false,"Killed":false,"Accumulables":[{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Update":1828,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":254,"Name":"internal.metrics.executorCpuTime","Update":18625831,"Value":98937761,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Update":38,"Value":127,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6238101,"Value":35849070,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":68,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6238101,"Executor Run Time":38,"Executor CPU Time":18625831,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":10,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":341644736,"JVMOffHeapMemory":103378144,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":541469,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":541469,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228406,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5011247104,"ProcessTreeJVMRSSMemory":658989056,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":10,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":228132872,"JVMOffHeapMemory":61634808,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":455614,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":455614,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20669,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3021172736,"ProcessTreeJVMRSSMemory":436867072,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026138,"Completion Time":1538441026376,"Accumulables":[{"ID":254,"Name":"internal.metrics.executorCpuTime","Value":98937761,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Value":127,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Value":35849070,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Value":3656,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Value":68,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1538441026376,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1538441026404,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[12,11],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026408,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":0,"Attempt":0,"Launch Time":1538441026450,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":1,"Attempt":0,"Launch Time":1538441026585,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":0,"Attempt":0,"Launch Time":1538441026450,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026586,"Failed":false,"Killed":false,"Accumulables":[{"ID":298,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Update":278446,"Value":278446,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Update":23317154,"Value":23317154,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Update":69,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Update":17832528,"Value":17832528,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Update":53,"Value":53,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":53,"Executor Deserialize CPU Time":17832528,"Executor Run Time":69,"Executor CPU Time":23317154,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":278446,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":1,"Attempt":0,"Launch Time":1538441026585,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026700,"Failed":false,"Killed":false,"Accumulables":[{"ID":298,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Update":215244,"Value":493690,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Update":23292541,"Value":46609695,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Update":94,"Value":163,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4400590,"Value":22233118,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":57,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4400590,"Executor Run Time":94,"Executor CPU Time":23292541,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":215244,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":11,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":198912952,"JVMOffHeapMemory":104016864,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":554933,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":554933,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5040721920,"ProcessTreeJVMRSSMemory":705302528,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":11,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":249428840,"JVMOffHeapMemory":62917480,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":455614,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":455614,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20911,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3035901952,"ProcessTreeJVMRSSMemory":447041536,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026408,"Completion Time":1538441026701,"Accumulables":[{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Value":22233118,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Value":57,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Value":46609695,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Value":163,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Value":493690,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026702,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":0,"Attempt":0,"Launch Time":1538441026714,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":1,"Attempt":0,"Launch Time":1538441026794,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":0,"Attempt":0,"Launch Time":1538441026714,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026795,"Failed":false,"Killed":false,"Accumulables":[{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Update":1871,"Value":1871,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.executorCpuTime","Update":16951615,"Value":16951615,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12613041,"Value":12613041,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Update":31,"Value":31,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":31,"Executor Deserialize CPU Time":12613041,"Executor Run Time":28,"Executor CPU Time":16951615,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":1,"Attempt":0,"Launch Time":1538441026794,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026839,"Failed":false,"Killed":false,"Accumulables":[{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Update":1871,"Value":3742,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.executorCpuTime","Update":17828037,"Value":34779652,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Update":24,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3879530,"Value":16492571,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":36,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3879530,"Executor Run Time":24,"Executor CPU Time":17828037,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":12,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":204287872,"JVMOffHeapMemory":104055736,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":519458,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":519458,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5047037952,"ProcessTreeJVMRSSMemory":708661248,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":12,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":252161344,"JVMOffHeapMemory":63019944,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":441078,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":441078,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20911,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3038007296,"ProcessTreeJVMRSSMemory":451837952,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026702,"Completion Time":1538441026840,"Accumulables":[{"ID":304,"Name":"internal.metrics.executorCpuTime","Value":34779652,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Value":36,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Value":52,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Value":16492571,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Value":3742,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1538441026840,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerUnpersistRDD","RDD ID":32} -{"Event":"SparkListenerUnpersistRDD","RDD ID":5} -{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1538441026935,"Stage Infos":[{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[13],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"79\",\"name\":\"sum\"}"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026936,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"79\",\"name\":\"sum\"}"}} -{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":0,"Attempt":0,"Launch Time":1538441026947,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Port":40992},"Timestamp":1538441027285} -{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"7","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000008 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000008\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":31,"Index":0,"Attempt":0,"Launch Time":1538441026947,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441027494,"Failed":true,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441027495,"Executor ID":"7","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000008 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000008\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538441032740,"Executor ID":"8","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000009/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000009/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":1,"Launch Time":1538441032741,"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Port":41485},"Maximum Memory":384093388,"Timestamp":1538441033142,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1538441036142,"Executor ID":"9","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stderr?start=-4096"}}} -{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1538441036144,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Port":40797},"Maximum Memory":384093388,"Timestamp":1538441036560,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Port":41485},"Timestamp":1538441040323} -{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"8","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000009 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000009\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":32,"Index":0,"Attempt":1,"Launch Time":1538441032741,"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441040533,"Failed":true,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441040534,"Executor ID":"8","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000009 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000009\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} -{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":0,"Attempt":2,"Launch Time":1538441042184,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1538441036144,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441042185,"Failed":false,"Killed":false,"Accumulables":[{"ID":348,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":331,"Name":"internal.metrics.jvmGCTime","Update":288,"Value":288,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Update":1539,"Value":1539,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Update":1278640624,"Value":1278640624,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Update":2796,"Value":2796,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Update":720112530,"Value":720112530,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Update":2587,"Value":2587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2587,"Executor Deserialize CPU Time":720112530,"Executor Run Time":2796,"Executor CPU Time":1278640624,"Result Size":1539,"JVM GC Time":288,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":36,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":0,"Attempt":2,"Launch Time":1538441042184,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441042334,"Failed":false,"Killed":false,"Accumulables":[{"ID":348,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Update":1453,"Value":2992,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Update":69678739,"Value":1348319363,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Update":118,"Value":2914,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6252896,"Value":726365426,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":2593,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6252896,"Executor Run Time":118,"Executor CPU Time":69678739,"Result Size":1453,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":266240264,"JVMOffHeapMemory":104976128,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":534126,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":534126,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5067235328,"ProcessTreeJVMRSSMemory":710475776,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"8","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":197860072,"JVMOffHeapMemory":57762424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1088805,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1088805,"OffHeapUnifiedMemory":0,"DirectPoolMemory":25453,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3028791296,"ProcessTreeJVMRSSMemory":430297088,"ProcessTreePythonVMemory":286212096,"ProcessTreePythonRSSMemory":30441472,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"9","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":193766856,"JVMOffHeapMemory":59006656,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1088805,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1088805,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20181,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016261632,"ProcessTreeJVMRSSMemory":405860352,"ProcessTreePythonVMemory":625926144,"ProcessTreePythonRSSMemory":69013504,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026936,"Completion Time":1538441042335,"Accumulables":[{"ID":331,"Name":"internal.metrics.jvmGCTime","Value":288,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Value":2914,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Value":726365426,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.input.recordsRead","Value":6,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Value":2992,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Value":2593,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Value":1348319363,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1538441042335,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1538441042338} diff --git a/core/src/test/resources/spark-events/application_1553914137147_0018 b/core/src/test/resources/spark-events/application_1553914137147_0018 new file mode 100644 index 000000000000..8c34e7265f8d --- /dev/null +++ b/core/src/test/resources/spark-events/application_1553914137147_0018 @@ -0,0 +1,44 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"test-1.vpc.company.com","Port":44768},"Maximum Memory":956615884,"Timestamp":1554755989747,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_144/jre","Java Version":"1.8.0_144 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.lineage.log.dir":"/var/log/spark2/lineage","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.yarn.jars":"local:/opt/cloudera/parcels/SPARK2/lib/spark2/jars/*","spark.executor.extraJavaOptions":"-Djava.security.egd=file:///dev/urandom","spark.driver.host":"test-1.vpc.company.com","spark.eventLog.enabled":"true","spark.executor.heartbeatInterval":"1000","spark.executor.memoryOverhead":"1024","spark.driver.port":"34194","spark.shuffle.service.enabled":"false","spark.driver.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.lineage.enabled":"false","spark.jars":"file:/tmp/__spark_test__/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar","spark.executor.metrics.pollingInterval":"100","spark.yarn.historyServer.address":"http://test-1.vpc.company.com:18089","spark.ui.enabled":"true","spark.app.name":"LargeBlocks","spark.ui.killEnabled":"true","spark.sql.hive.metastore.jars":"${env:HADOOP_COMMON_HOME}/../hive/lib/*:${env:HADOOP_COMMON_HOME}/client/*","spark.locality.wait.process":"0","spark.dynamicAllocation.schedulerBacklogTimeout":"1","spark.yarn.am.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.scheduler.mode":"FIFO","spark.eventLog.logStageExecutorMetrics.enabled":"true","spark.driver.memory":"2g","spark.executor.instances":"3","spark.submit.pyFiles":"","spark.yarn.config.gatewayPath":"/opt/cloudera/parcels","spark.executor.id":"driver","spark.yarn.config.replacementPath":"{{HADOOP_COMMON_HOME}}/../../..","spark.driver.extraJavaOptions":"-Djava.security.egd=file:///dev/urandom","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.submit.deployMode":"client","spark.shuffle.service.port":"7337","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.authenticate":"false","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.network.crypto.enabled":"false","spark.executor.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.executor.memory":"7g","spark.io.encryption.enabled":"false","spark.eventLog.dir":"hdfs://test-1.vpc.company.com:8020/user/spark/spark2ApplicationHistory","spark.dynamicAllocation.enabled":"false","spark.sql.catalogImplementation":"hive","spark.executor.cores":"1","spark.driver.appUIAddress":"http://test-1.vpc.company.com:4040","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"test-1.vpc.company.com","spark.dynamicAllocation.minExecutors":"0","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://test-1.vpc.company.com:8088/proxy/application_1553914137147_0018","spark.dynamicAllocation.executorIdleTimeout":"60","spark.app.id":"application_1553914137147_0018","spark.sql.hive.metastore.version":"1.1.0"},"Hadoop Properties":{"yarn.resourcemanager.amlauncher.thread-count":"50","dfs.namenode.resource.check.interval":"5000","fs.s3a.connection.maximum":"100","mapreduce.jobtracker.jobhistory.task.numberprogresssplits":"12","dfs.data.transfer.server.tcpnodelay":"true","mapreduce.tasktracker.healthchecker.script.timeout":"600000","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.jobhistory.loadedjob.tasks.max":"-1","mapreduce.framework.name":"yarn","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","dfs.cachereport.intervalMsec":"10000","dfs.namenode.checkpoint.txns":"1000000","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","mapreduce.tasktracker.local.dir.minspacekill":"0","hadoop.security.groups.cache.background.reload.threads":"3","dfs.namenode.lease-recheck-interval-ms":"2000","fs.s3.block.size":"67108864","dfs.client.domain.socket.data.traffic":"false","dfs.ha.zkfc.nn.http.timeout.ms":"20000","hadoop.registry.secure":"false","hadoop.hdfs.configuration.version":"1","dfs.bytes-per-checksum":"512","fs.s3.buffer.dir":"${hadoop.tmp.dir}/s3","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25","mapreduce.jobhistory.loadedjobs.cache.size":"5","mapreduce.jobtracker.persist.jobstatus.hours":"1","fs.s3a.s3guard.ddb.table.create":"false","dfs.datanode.slow.io.warning.threshold.ms":"300","dfs.namenode.handler.count":"10","dfs.namenode.list.reencryption.status.num.responses":"100","mapreduce.input.fileinputformat.split.minsize":"0","dfs.datanode.failed.volumes.tolerated":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.amliveliness-monitor.interval-ms":"1000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","mapreduce.tasktracker.http.threads":"40","dfs.namenode.retrycache.expirytime.millis":"600000","dfs.namenode.backup.address":"0.0.0.0:50100","dfs.datanode.data.dir":"file://${hadoop.tmp.dir}/dfs/data","dfs.datanode.shared.file.descriptor.paths":"/dev/shm,/tmp","dfs.replication":"3","mapreduce.jobtracker.jobhistory.block.size":"3145728","dfs.encrypt.data.transfer.cipher.key.bitlength":"128","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","dfs.secondary.namenode.kerberos.internal.spnego.principal":"${dfs.web.authentication.kerberos.principal}","mapreduce.task.profile.maps":"0-2","dfs.datanode.block-pinning.enabled":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","mapreduce.jobtracker.retiredjobs.cache.size":"1000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"test-1.vpc.company.com:19890","fs.trash.checkpoint.interval":"0","dfs.namenode.checkpoint.check.period":"60","yarn.nodemanager.container-monitor.interval-ms":"3000","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","hadoop.proxyuser.HTTP.groups":"*","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"10000","dfs.namenode.reencrypt.edek.threads":"10","yarn.resourcemanager.system-metrics-publisher.enabled":"false","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","dfs.namenode.fs-limits.max-component-length":"255","mapreduce.tasktracker.taskcontroller":"org.apache.hadoop.mapred.DefaultTaskController","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","dfs.namenode.top.window.num.buckets":"10","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","dfs.datanode.block.id.layout.upgrade.threads":"12","mapreduce.jobtracker.tasktracker.maxblacklists":"4","yarn.nodemanager.docker-container-executor.exec-name":"/usr/bin/docker","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"0.23.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","dfs.namenode.decommission.max.concurrent.tracked.nodes":"100","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","dfs.client.read.shortcircuit.skip.checksum":"false","mapreduce.task.profile.reduces":"0-2","dfs.datanode.address":"0.0.0.0:50010","dfs.https.server.keystore.resource":"ssl-server.xml","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"test-1.vpc.company.com:8030","mapreduce.task.timeout":"600000","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"hdfs://test-1.vpc.company.com:8020","fs.har.impl.disable.cache":"true","io.compression.codec.bzip2.library":"system-native","dfs.namenode.audit.loggers":"default","dfs.block.access.key.update.interval":"600","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"test-1.vpc.company.com:8090","dfs.namenode.max.objects":"0","mapreduce.jobhistory.address":"test-1.vpc.company.com:10020","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","dfs.namenode.max.full.block.report.leases":"6","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","dfs.client.mmap.cache.timeout.ms":"3600000","dfs.mover.max-no-move-interval":"60000","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","dfs.client.datanode-restart.timeout":"30","dfs.datanode.drop.cache.behind.reads":"false","ipc.server.log.slow.rpc":"false","dfs.namenode.read-lock-reporting-threshold-ms":"5000","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","yarn.nodemanager.default-container-executor.log-dirs.permissions":"710","dfs.namenode.checkpoint.edits.dir":"${dfs.namenode.checkpoint.dir}","yarn.app.attempt.diagnostics.limit.kc":"64","dfs.balancer.block-move.timeout":"0","dfs.client.block.write.replace-datanode-on-failure.enable":"true","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"8192","io.compression.codecs":"org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.DeflateCodec,org.apache.hadoop.io.compress.SnappyCodec,org.apache.hadoop.io.compress.Lz4Codec","fs.s3a.fast.upload.active.blocks":"4","hadoop.security.credential.clear-text-fallback":"true","dfs.heartbeat.interval":"3","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","dfs.ha.tail-edits.period":"60","dfs.datanode.max.locked.memory":"0","dfs.datanode.scan.period.hours":"504","mapreduce.jobtracker.expire.trackers.interval":"600000","yarn.resourcemanager.nodemanager-connect-retries":"10","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","dfs.namenode.edits.noeditlogchannelflush":"false","mapreduce.task.io.sort.factor":"64","mapreduce.tasktracker.outofband.heartbeat":"false","ha.failover-controller.new-active.rpc-timeout.ms":"60000","dfs.webhdfs.ugi.expire.after.access":"600000","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","dfs.namenode.https-address":"test-1.vpc.company.com:20102","yarn.am.blacklisting.enabled":"true","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","dfs.image.transfer-bootstrap-standby.bandwidthPerSec":"0","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.proxyuser.hive.groups":"*","ha.zookeeper.session-timeout.ms":"5000","mapreduce.map.java.opts":"-Djava.net.preferIPv4Stack=true","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","dfs.ha.automatic-failover.enabled":"false","mapreduce.job.counters.groups.max":"50","dfs.namenode.decommission.interval":"30","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","dfs.datanode.handler.count":"10","dfs.namenode.xattrs.enabled":"true","dfs.namenode.safemode.threshold-pct":"0.999f","dfs.client.socket.send.buffer.size":"0","mapreduce.map.sort.spill.percent":"0.8","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","dfs.datanode.sync.behind.writes":"false","dfs.namenode.stale.datanode.interval":"30000","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","dfs.datanode.transfer.socket.send.buffer.size":"0","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","dfs.namenode.reencrypt.throttle.limit.handler.ratio":"1.0","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler","yarn.app.mapreduce.am.command-opts":"-Djava.net.preferIPv4Stack=true -Xmx825955249","dfs.journalnode.https-address":"0.0.0.0:8481","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","hadoop.proxyuser.hue.hosts":"*","io.mapfile.bloom.error.rate":"0.005","dfs.user.home.dir.prefix":"/user","hadoop.proxyuser.hue.groups":"*","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","mapreduce.jobtracker.persist.jobstatus.dir":"/jobtracker/jobsInfo","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","fs.s3a.metadatastore.impl":"org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore","io.skip.checksum.errors":"false","dfs.namenode.snapshot.capture.openfiles":"false","dfs.datanode.directoryscan.interval":"21600","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","dfs.client.read.shortcircuit.streams.cache.expiry.ms":"300000","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"10","dfs.namenode.write.stale.datanode.ratio":"0.5f","hadoop.registry.zk.session.timeout.ms":"60000","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","mapreduce.jobtracker.taskcache.levels":"2","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","mapreduce.job.emit-timeline-data":"false","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","dfs.domain.socket.path":"/var/run/hdfs-sockets/dn","yarn.resourcemanager.admin.client.thread-count":"1","mapreduce.jobtracker.persist.jobstatus.active":"true","dfs.namenode.reencrypt.sleep.interval":"1m","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"test-1.vpc.company.com:10033","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","mapreduce.tasktracker.report.address":"127.0.0.1:0","dfs.namenode.edit.log.autoroll.multiplier.threshold":"2.0","io.seqfile.lazydecompress":"true","ftp.blocksize":"67108864","dfs.namenode.backup.http-address":"0.0.0.0:50105","dfs.disk.balancer.max.disk.throughputInMBperSec":"10","mapreduce.jobtracker.instrumentation":"org.apache.hadoop.mapred.JobTrackerMetricsInst","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.delete.debug-delay-sec":"0","mapreduce.jobtracker.http.address":"0.0.0.0:50030","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","hadoop.security.groups.cache.secs":"300","yarn.resourcemanager.zk-retry-interval-ms":"1000","yarn.scheduler.increment-allocation-mb":"512","nfs.mountd.port":"4242","mapreduce.shuffle.max.threads":"0","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","mapreduce.jobhistory.http.policy":"HTTP_ONLY","dfs.datanode.dns.interface":"default","mapreduce.reduce.java.opts":"-Djava.net.preferIPv4Stack=true","s3native.replication":"3","hadoop.security.group.mapping.ldap.ssl":"false","dfs.namenode.fs-limits.max-xattrs-per-inode":"32","yarn.client.application-client-protocol.poll-interval-ms":"200","hadoop.proxyuser.flume.groups":"*","dfs.namenode.fs-limits.max-xattr-size":"16384","dfs.namenode.maintenance.replication.min":"1","dfs.client.write.exclude.nodes.cache.expiry.interval.millis":"600000","ha.zookeeper.parent-znode":"/hadoop-ha","dfs.namenode.safemode.extension":"30000","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","dfs.blocksize":"134217728","dfs.namenode.servicerpc-address":"test-1.vpc.company.com:8022","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"test-1.vpc.company.com:8032","ipc.client.ping":"true","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","dfs.namenode.resource.checked.volumes.minimum":"1","hadoop.proxyuser.HTTP.hosts":"*","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","dfs.namenode.quota.init-threads":"4","dfs.journalnode.http-address":"0.0.0.0:8480","dfs.block.scanner.volume.bytes.per.second":"1048576","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","dfs.storage.policy.enabled":"true","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","dfs.namenode.edits.dir":"${dfs.namenode.name.dir}","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","dfs.namenode.support.allow.format":"true","dfs.ha.tail-edits.rolledits.timeout":"60","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","mapreduce.cluster.temp.dir":"${hadoop.tmp.dir}/mapred/temp","s3.replication":"3","dfs.client.failover.connection.retries":"0","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","dfs.namenode.secondary.http-address":"0.0.0.0:50090","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","nfs.wtmax":"1048576","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","nfs.dump.dir":"/tmp/.hdfs-nfs","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","mapreduce.client.submit.file.replication":"3","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"128M","dfs.namenode.service.handler.count":"10","dfs.datanode.data.dir.perm":"700","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","dfs.namenode.name.dir":"file:///dataroot/dataroot/dfs/nn","yarn.resourcemanager.zk-acl":"world:anyone:rwcda","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","fs.ftp.host.port":"21","ipc.ping.interval":"60000","dfs.namenode.num.checkpoints.retained":"2","dfs.namenode.kerberos.internal.spnego.principal":"${dfs.web.authentication.kerberos.principal}","yarn.resourcemanager.admin.address":"test-1.vpc.company.com:8033","file.client-write-packet-size":"65536","hadoop.treat.subject.external":"false","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"false","dfs.disk.balancer.plan.threshold.percent":"10","mapreduce.local.clientfactory.class.name":"org.apache.hadoop.mapred.LocalClientFactory","dfs.client.use.legacy.blockreader":"false","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"256","yarn.nodemanager.localizer.client.thread-count":"5","dfs.namenode.checkpoint.max-retries":"3","dfs.namenode.reject-unresolved-dn-topology-mapping":"false","dfs.namenode.delegation.token.max-lifetime":"*********(redacted)","dfs.client.block.write.replace-datanode-on-failure.min-replication":"0","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec,org.apache.hadoop.crypto.JceAesCtrCryptoCodec","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"2000","dfs.namenode.num.extra.edits.retained":"1000000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","yarn.scheduler.increment-allocation-vcores":"1","fs.df.interval":"60000","fs.s3.sleepTimeSeconds":"10","fs.s3a.multiobjectdelete.enable":"true","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"true","io.file.buffer.size":"65536","hadoop.work.around.non.threadsafe.getpwuid":"false","dfs.permissions.superusergroup":"supergroup","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","mapreduce.tasktracker.dns.interface":"default","hadoop.security.sensitive-config-keys":"*********(redacted)","fs.s3a.s3guard.ddb.max.retries":"9","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","dfs.permissions.enabled":"true","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/user","mapreduce.reduce.shuffle.read.timeout":"180000","yarn.app.mapreduce.am.admin.user.env":"LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH","dfs.datanode.https.address":"0.0.0.0:50475","dfs.namenode.hosts.provider.classname":"org.apache.hadoop.hdfs.server.blockmanagement.HostFileManager","dfs.datanode.transfer.socket.recv.buffer.size":"0","fs.s3a.connection.establish.timeout":"5000","dfs.namenode.fslock.fair":"true","mapreduce.job.running.map.limit":"0","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","hadoop.fuse.connection.timeout":"300","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","adl.http.timeout":"5000","yarn.nodemanager.vmem-pmem-ratio":"2.1","dfs.client.slow.io.warning.threshold.ms":"30000","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","s3native.stream-buffer-size":"4096","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","fs.s3a.s3guard.cli.prune.age":"86400000","dfs.client.read.shortcircuit.streams.cache.size":"256","dfs.client.use.legacy.blockreader.local":"false","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3n.multipart.uploads.enabled":"false","dfs.namenode.path.based.cache.retry.interval.ms":"30000","hadoop.security.crypto.buffer.size":"8192","yarn.client.failover-retries-on-socket-timeouts":"0","dfs.balancer.keytab.enabled":"false","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","dfs.datanode.balance.bandwidthPerSec":"10485760","dfs.namenode.name.dir.restore":"false","hadoop.registry.jaas.context":"Client","dfs.client.failover.sleep.max.millis":"15000","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","s3.blocksize":"67108864","yarn.am.blacklisting.disable-failure-threshold":"0.8f","io.map.index.interval":"128","mapreduce.job.counters.max":"120","dfs.namenode.max-lock-hold-to-release-lease-ms":"25","dfs.namenode.datanode.registration.ip-hostname-check":"true","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","dfs.namenode.resource.du.reserved":"104857600","dfs.datanode.bp-ready.timeout":"20","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","dfs.namenode.full.block.report.lease.length.ms":"300000","mapreduce.tasktracker.instrumentation":"org.apache.hadoop.mapred.TaskTrackerMetricsInst","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","s3.stream-buffer-size":"4096","yarn.resourcemanager.nm.liveness-monitor.interval-ms":"1000","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","dfs.namenode.secondary.https-address":"0.0.0.0:50091","s3native.bytes-per-checksum":"512","dfs.namenode.fs-limits.max-directory-items":"1048576","nfs.server.port":"2049","dfs.namenode.delegation.token.renew-interval":"*********(redacted)","mapreduce.jobtracker.address":"local","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","dfs.namenode.list.openfiles.num.responses":"1000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","dfs.namenode.blocks.per.postponedblocks.rescan":"10000","dfs.namenode.checkpoint.period":"3600","hadoop.security.groups.cache.background.reload":"false","yarn.resourcemanager.amlauncher.log.command":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","dfs.namenode.snapshot.skip.capture.accesstime-only-change":"false","dfs.webhdfs.user.provider.user.pattern":"^[A-Za-z_][A-Za-z0-9._-]*[$]?$","dfs.webhdfs.acl.provider.permission.pattern":"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$","mapreduce.fileoutputcommitter.algorithm.version":"1","yarn.resourcemanager.work-preserving-recovery.enabled":"false","mapreduce.map.skip.maxrecords":"0","mapreduce.jobtracker.handler.count":"10","hadoop.http.authentication.type":"simple","mapreduce.job.jvm.numtasks":"1","hadoop.proxyuser.flume.hosts":"*","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","fs.s3n.block.size":"67108864","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","mapreduce.jobtracker.staging.root.dir":"${hadoop.tmp.dir}/mapred/staging","dfs.namenode.http-address":"test-1.vpc.company.com:20101","mapreduce.jobtracker.jobhistory.lru.cache.size":"5","dfs.datanode.directoryscan.threads":"1","dfs.datanode.fsdatasetcache.max.threads.per.volume":"4","dfs.namenode.fs-limits.max-blocks-per-file":"1048576","dfs.disk.balancer.enabled":"false","mapreduce.shuffle.listen.queue.size":"128","mapreduce.tasktracker.local.dir.minspacestart":"0","mapreduce.map.cpu.vcores":"1","hadoop.user.group.static.mapping.overrides":"dr.who=;","dfs.datanode.cache.revocation.timeout.ms":"900000","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","dfs.client.mmap.cache.size":"256","dfs.ha.log-roll.period":"120","dfs.client.failover.sleep.base.millis":"500","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","dfs.namenode.accesstime.precision":"3600000","yarn.app.mapreduce.client.job.max-retries":"3","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","hadoop.registry.zk.retry.interval.ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","dfs.client.context":"default","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","hadoop.ssl.server.conf":"ssl-server.xml","dfs.http.policy":"HTTP_ONLY","dfs.client.https.keystore.resource":"ssl-client.xml","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.acl.enable":"true","dfs.domain.socket.disable.interval.seconds":"1","dfs.image.transfer.chunksize":"65536","dfs.balancer.max-no-move-interval":"60000","mapreduce.tasktracker.map.tasks.maximum":"2","dfs.namenode.edits.journal-plugin.qjournal":"org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager","mapreduce.task.profile":"false","dfs.webhdfs.enabled":"true","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","dfs.namenode.list.encryption.zones.num.responses":"100","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","dfs.namenode.top.num.users":"10","dfs.disk.balancer.block.tolerance.percent":"10","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","io.seqfile.sorter.recordlimit":"1000000","hadoop.security.auth_to_local":"DEFAULT","dfs.blockreport.initialDelay":"0","fs.automatic.close":"true","dfs.client.block.write.replace-datanode-on-failure.best-effort":"false","dfs.namenode.replication.min":"1","dfs.balancer.address":"0.0.0.0:0","fs.s3n.multipart.copy.block.size":"5368709120","yarn.nodemanager.hostname":"0.0.0.0","nfs.rtmax":"1048576","yarn.resourcemanager.zk-timeout-ms":"10000","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","dfs.datanode.directoryscan.throttle.limit.ms.per.sec":"1000","yarn.nodemanager.container-localizer.log.level":"INFO","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","dfs.namenode.replication.work.multiplier.per.iteration":"2","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","dfs.namenode.reencrypt.throttle.limit.updater.ratio":"1.0","dfs.namenode.avoid.write.stale.datanode":"false","dfs.short.circuit.shared.memory.watcher.interrupt.check.ms":"60000","dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction":"0.75f","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","dfs.client.mmap.enabled":"true","mapreduce.reduce.cpu.vcores":"1","hadoop.proxyuser.oozie.groups":"*","fs.client.resolve.remote.symlinks":"true","dfs.image.compression.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.jobtracker.restart.recover":"false","dfs.namenode.decommission.blocks.per.interval":"500000","mapreduce.tasktracker.reduce.tasks.maximum":"2","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","dfs.namenode.safemode.min.datanodes":"0","fs.ftp.data.connection.mode":"ACTIVE_LOCAL_DATA_CONNECTION_MODE","mapreduce.job.userlog.retain.hours":"24","yarn.scheduler.maximum-allocation-vcores":"4","yarn.nodemanager.log-aggregation.compression-type":"none","dfs.namenode.enable.retrycache":"true","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","dfs.namenode.startup.delay.block.deletion.sec":"0","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","dfs.datanode.readahead.bytes":"4194304","mapreduce.jobtracker.heartbeats.in.second":"100","mapreduce.job.running.reduce.limit":"0","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","mapreduce.task.tmp.dir":"./tmp","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:mapred@hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","dfs.namenode.delegation.key.update-interval":"86400000","fs.s3a.max.total.tasks":"5","dfs.client.file-block-storage-locations.num-threads":"10","mapreduce.tasktracker.healthchecker.interval":"60000","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.fast.upload":"false","fs.s3a.attempts.maximum":"20","dfs.namenode.avoid.read.stale.datanode":"false","hadoop.registry.zk.connection.timeout.ms":"15000","dfs.https.port":"20102","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","mapreduce.map.log.level":"INFO","mapreduce.output.fileoutputformat.compress.type":"BLOCK","hadoop.registry.rm.enabled":"false","mapreduce.ifile.readahead.bytes":"4194304","mapreduce.tasktracker.tasks.sleeptimebeforesigkill":"5000","yarn.resourcemanager.fs.state-store.retry-policy-spec":"2000, 500","dfs.namenode.posix.acl.inheritance.enabled":"false","dfs.blockreport.intervalMsec":"21600000","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","dfs.namenode.path.based.cache.refresh.interval.ms":"30000","dfs.namenode.edekcacheloader.interval.ms":"1000","file.stream-buffer-size":"4096","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.SnappyCodec","mapreduce.map.speculative":"false","dfs.disk.balancer.max.disk.errors":"5","dfs.datanode.use.datanode.hostname":"false","mapreduce.job.speculative.retry-after-speculate":"15000","hadoop.proxyuser.hdfs.hosts":"*","dfs.namenode.fs-limits.min-block-size":"1048576","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","mapreduce.job.reduce.slowstart.completedmaps":"0.8","dfs.client.read.shortcircuit":"false","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","hadoop.proxyuser.yarn.groups":"*","dfs.client.cached.conn.retry":"3","dfs.namenode.invalidate.work.pct.per.iteration":"0.32f","hadoop.http.logs.enabled":"true","fs.s3a.block.size":"32M","yarn.nodemanager.logaggregation.threadpool-size-max":"100","dfs.replication.max":"512","dfs.namenode.inotify.max.events.per.rpc":"1000","yarn.resourcemanager.hostname":"0.0.0.0","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"0","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","ipc.client.rpc-timeout.ms":"0","fs.s3.maxRetries":"4","dfs.default.chunk.view.size":"32768","mapreduce.input.lineinputformat.linespermap":"1","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","dfs.client.mmap.retry.timeout.ms":"300000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","dfs.namenode.list.cache.directives.num.responses":"100","fs.s3a.socket.recv.buffer":"8192","dfs.image.compress":"false","dfs.namenode.kerberos.principal.pattern":"*","yarn.application.classpath":"$HADOOP_CLIENT_CONF_DIR,$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_YARN_HOME/*,$HADOOP_YARN_HOME/lib/*","fs.s3n.multipart.uploads.block.size":"67108864","mapreduce.tasktracker.http.address":"0.0.0.0:50060","yarn.resourcemanager.resource-tracker.address":"test-1.vpc.company.com:8031","hadoop.fuse.timer.period":"5","mapreduce.job.heap.memory-mb.ratio":"0.8","dfs.datanode.hdfs-blocks-metadata.enabled":"true","dfs.namenode.checkpoint.dir":"file://${hadoop.tmp.dir}/dfs/namesecondary","dfs.datanode.max.transfer.threads":"4096","dfs.namenode.edits.asynclogging":"true","nfs.allow.insecure.ports":"true","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"0","s3native.client-write-packet-size":"65536","mapreduce.admin.user.env":"LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","hadoop.proxyuser.mapred.hosts":"*","hadoop.proxyuser.oozie.hosts":"*","yarn.nodemanager.log.retain-seconds":"10800","hadoop.proxyuser.mapred.groups":"*","yarn.resourcemanager.keytab":"/etc/krb5.keytab","mapreduce.reduce.merge.inmem.threshold":"1000","dfs.client.https.need-auth":"false","dfs.blockreport.split.threshold":"1000000","dfs.client.block.write.replace-datanode-on-failure.policy":"DEFAULT","mapreduce.shuffle.ssl.enabled":"false","dfs.namenode.write-lock-reporting-threshold-ms":"5000","dfs.block.access.token.enable":"*********(redacted)","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","httpfs.buffer.size":"4096","dfs.client.file-block-storage-locations.timeout.millis":"1000","dfs.namenode.block-placement-policy.default.prefer-local-node":"true","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","dfs.namenode.replication.considerLoad":"true","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.proxyuser.hdfs.groups":"*","dfs.namenode.retrycache.heap.percent":"0.03f","dfs.datanode.cache.revocation.polling.ms":"500","mapreduce.jobhistory.webapp.address":"test-1.vpc.company.com:19888","dfs.namenode.path.based.cache.block.map.allocation.percent":"0.25","mapreduce.jobtracker.system.dir":"${hadoop.tmp.dir}/mapred/system","mapreduce.tasktracker.taskmemorymanager.monitoringinterval":"5000","dfs.journalnode.rpc-address":"0.0.0.0:8485","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"test-1.vpc.company.com:8088","mapreduce.jobhistory.recovery.enable":"false","dfs.client.short.circuit.replica.stale.threshold.ms":"1800000","mapreduce.reduce.shuffle.parallelcopies":"10","fs.trash.interval":"1","dfs.namenode.replication.interval":"3","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","dfs.namenode.top.enabled":"true","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","dfs.datanode.du.reserved":"0","yarn.app.mapreduce.am.resource.mb":"1024","mapreduce.input.fileinputformat.list-status.num-threads":"1","dfs.namenode.lazypersist.file.scrub.interval.sec":"300","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.nodemanager.resource.cpu-vcores":"8","mapreduce.job.reduces":"6","fs.s3a.multipart.size":"64M","yarn.scheduler.minimum-allocation-vcores":"1","dfs.namenode.reencrypt.batch.size":"1000","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","dfs.datanode.http.address":"0.0.0.0:50075","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","fs.s3a.metadatastore.authoritative":"false","ha.health-monitor.sleep-after-disconnect.ms":"1000","s3.bytes-per-checksum":"512","yarn.app.mapreduce.shuffle.log.limit.kb":"0","dfs.namenode.list.cache.pools.num.responses":"100","hadoop.security.group.mapping":"org.apache.hadoop.security.ShellBasedUnixGroupsMapping","mapreduce.jobhistory.jhist.format":"binary","yarn.resourcemanager.ha.enabled":"false","dfs.encrypt.data.transfer":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","mapreduce.job.redacted-properties":"*********(redacted)","dfs.namenode.top.windows.minutes":"1,5,25","s3.client-write-packet-size":"65536","mapreduce.map.output.compress":"true","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.scheduler.maximum-allocation-mb":"12288","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.app.mapreduce.am.container.log.limit.kb":"0","s3native.blocksize":"67108864","ipc.client.connect.retry.interval":"1000","hadoop.proxyuser.httpfs.groups":"*","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","dfs.namenode.edit.log.autoroll.check.interval.ms":"300000","mapreduce.jobhistory.cleaner.enable":"true","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","hadoop.proxyuser.httpfs.hosts":"*","dfs.client.use.datanode.hostname":"false","dfs.stream-buffer-size":"4096","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","dfs.datanode.drop.cache.behind.writes":"false","mapreduce.tasktracker.dns.nameserver":"default","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","mapreduce.job.end-notification.retry.attempts":"0","hadoop.proxyuser.yarn.hosts":"*","yarn.resourcemanager.zk-num-retries":"1000","dfs.client.failover.max.attempts":"15","mapreduce.tasktracker.indexcache.mb":"10","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","dfs.namenode.snapshotdiff.allow.snap-root-descendant":"true","yarn.nodemanager.localizer.cache.target-size-mb":"10240","zlib.compress.level":"DEFAULT_COMPRESSION","ftp.client-write-packet-size":"65536","mapreduce.jobtracker.maxtasks.perjob":"-1","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","hadoop.proxyuser.hive.hosts":"*","dfs.block.access.token.lifetime":"*********(redacted)","dfs.namenode.max.extra.edits.segments.retained":"10000","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","dfs.image.transfer.bandwidthPerSec":"0","io.native.lib.available":"true","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","ipc.server.listen.queue.size":"128","dfs.namenode.edekcacheloader.initial.delay.ms":"3000","map.sort.class":"org.apache.hadoop.util.QuickSort","dfs.namenode.acls.enabled":"false","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","dfs.datanode.ipc.address":"0.0.0.0:50020","yarn.nodemanager.vmem-check-enabled":"false","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","dfs.lock.suppress.warning.interval":"10s","dfs.client.block.write.retries":"3","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.application.classpath":"$HADOOP_MAPRED_HOME/*,$HADOOP_MAPRED_HOME/lib/*,$MR2_CLASSPATH","yarn.client.nodemanager-connect.retry-interval-ms":"10000","dfs.client-write-packet-size":"65536","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,HADOOP_YARN_HOME","dfs.datanode.dns.nameserver":"default","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","dfs.image.transfer.timeout":"60000","yarn.resourcemanager.recovery.enabled":"false","dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold":"10737418240","dfs.client.failover.connection.retries.on.timeouts":"0"},"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.security.egd":"file:///dev/urandom","java.vm.specification.version":"1.8","user.home":"/home/systest","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_144/jre/lib/amd64","user.dir":"/tmp","java.library.path":":/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.144-b01","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/java/jdk1.8.0_144/jre/lib/endorsed","java.runtime.version":"1.8.0_144-b01","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_144/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_144/jre/lib/resources.jar:/usr/java/jdk1.8.0_144/jre/lib/rt.jar:/usr/java/jdk1.8.0_144/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_144/jre/lib/jsse.jar:/usr/java/jdk1.8.0_144/jre/lib/jce.jar:/usr/java/jdk1.8.0_144/jre/lib/charsets.jar:/usr/java/jdk1.8.0_144/jre/lib/jfr.jar:/usr/java/jdk1.8.0_144/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-514.26.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"systest","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --conf spark.driver.memory=2g --conf spark.executor.heartbeatInterval=1000 --conf spark.executor.metrics.pollingInterval=100 --conf spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true --conf spark.yarn.maxAppAttempts=1 --conf spark.locality.wait.process=0 --conf spark.executor.memoryOverhead=1024 --conf spark.executor.extraJavaOptions=-Djava.security.egd=file:///dev/urandom --conf spark.eventLog.logStageExecutorMetrics.enabled=true --conf spark.driver.extraJavaOptions=-Djava.security.egd=file:///dev/urandom --class com.company.spark.LargeBlocks --num-executors 3 --executor-memory 7g /tmp/__spark_test__/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar --targetBlockSizeGb 2.5 --taskSleepMillis 200 --doCache true --cacheOnDisk true --replicas 1 --concurrentReadJobs 2","java.home":"/usr/java/jdk1.8.0_144/jre","java.version":"1.8.0_144","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/opt/cloudera/parcels/CDH/jars/jackson-mapper-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/joni-2.1.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-core-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-external-blockcache-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-json-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/xz-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-annotations-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-procedure-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/asm-3.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xml-apis-1.3.04.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-jvm-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-dbcp-1.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-xml-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/minlog-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hsqldb-1.8.0.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/pmml-model-1.4.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spire-macros_2.12-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/httpcore-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jta-1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-logging-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/activation-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xbean-asm7-shaded-4.12.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/paranamer-2.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/lib/hadoop/NOTICE.txt":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-format-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/httpclient-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/netty-3.9.9.Final.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-nativetask-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-client-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-xc-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-configuration-1.6.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-math3-3.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jsp-api-2.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-auth-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-registry-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/JavaEWAH-0.3.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-graphite-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/joda-time-2.9.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-gridmix-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-compress-1.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javolution-5.5.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-beanutils-1.7.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/core-1.1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr-runtime-3.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-hdfs-nfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-hdfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-net-3.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/snappy-java-1.1.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-assembly_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-examples-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/shapeless_2.12-2.3.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-hs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-server-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-lang-2.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jtransforms-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/snappy-0.2.jar":"System Classpath","/etc/spark2/conf/yarn-conf/":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-cli-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-core-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.annotation-api-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-util-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stax-api-1.0-2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javassist-3.18.1-GA.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-applicationhistoryservice-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/kafka-clients-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/guice-3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr4-runtime-4.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/core-3.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/opencsv-2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-datajoin-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-common-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jdo-api-3.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-webapp-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-compiler-3.0.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-annotations-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/libthrift-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/kafka_2.11-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-azure-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-proxy-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr-2.7.7.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jettison-1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-core-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/libfb303-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/zookeeper-3.4.5-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-resource-bundle-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-streaming-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-hadoop2-compat-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/pyrolite-4.13.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/activation-1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aircompressor-0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-ant-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jline-2.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/netty-all-4.0.23.Final.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-servlets-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-xml_2.12-1.0.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-hs-plugins-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-web-proxy-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-math-2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-1.8.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/objenesis-2.5.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stream-2.7.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-aws-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-module-scala_2.12-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/spymemcached-2.11.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-core-1.5.5-nohive.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/mockito-all-1.8.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jets3t-0.9.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-xc-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-rsgroup-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/py4j-0.10.8.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-applications-distributedshell-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-rumen-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsch-0.1.42.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/zkclient-0.7.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xmlenc-0.52.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-io-2.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-core-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jamon-runtime-2.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-common-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-common-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/netty-3.10.5.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-app-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/machinist_2.12-0.6.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-util-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/chill_2.12-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/findbugs-annotations-1.3.9-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-rest-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-hadoop-compat-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/ST4-4.0.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jruby-cloudera-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-distcp-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-server-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/htrace-core-3.2.0-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-mapreduce-1.5.5-nohive.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-digester-1.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-http-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-math3-3.4.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/slf4j-log4j12-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-shell-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-server-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/chill-java-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stax-api-1.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsp-api-2.1-6.1.14.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/log4j-1.2.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.inject-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jasper-compiler-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-security-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-shuffle-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-nfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/netty-all-4.1.30.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jaxb-api-2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/breeze-macros_2.12-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jodd-core-3.5.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/azure-data-lake-store-sdk-2.2.9.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/guava-12.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-codec-1.10.jar":"System Classpath","/usr/java/jdk1.8.0_144/lib/tools.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-nodemanager-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jcodings-1.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-azure-datalake-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/guava-11.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-io-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-applications-unmanaged-am-launcher-2.6.0-cdh5.15.2.jar":"System Classpath","/etc/spark2/conf/":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/httpclient-4.5.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-continuation-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-ast_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-jndi-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsr305-3.0.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/avro-1.7.6-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/univocity-parsers-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-annotations-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jettison-1.3.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hppc-0.7.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/logredactor-1.0.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-vector-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jaxb-impl-2.2.3-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-archives-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-plus-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-client-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hue-plugins-3.9.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/high-scale-lib-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-library-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/java-xmlbuilder-0.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-format-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/paranamer-2.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spire_2.12-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/disruptor-3.3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-pool-1.5.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-servlet-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arpack_combined_all-0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-annotations-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsp-2.1-6.1.14.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/log4j-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/derby-10.12.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-shims-1.5.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-daemon-1.0.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/ivy-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hamcrest-core-1.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-lang3-3.8.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/spark-streaming-kafka-0-8_2.11-2.4.0.cloudera1-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-databind-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-reflect-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-examples-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/httpcore-4.4.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-memory-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/lib/hadoop/LICENSE.txt":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-compress-1.8.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-client-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/oro-2.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-thrift-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-ipc-1.8.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/okhttp-2.4.0.jar":"System Classpath","spark://test-1.vpc.company.com:34194/jars/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar":"Added By User","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-compiler-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-prefix-tree-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-api-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-codec-1.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/lz4-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/janino-3.0.11.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/libthrift-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-core_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/lz4-java-1.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-jaxrs-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/breeze_2.12-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-sls-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/snappy-java-1.0.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-guava-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/validation-api-1.1.0.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/compress-lzf-1.0.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-openstack-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-client-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-it-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-extras-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-column-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/unused-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-crypto-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-databind-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-codec-1.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/guava-14.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/aws-java-sdk-bundle-1.11.134.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/zookeeper-3.4.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-module-paranamer-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-protocol-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jasper-runtime-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-logging-1.1.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-resourcemanager-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-archive-logs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/paranamer-2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/metrics-core-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-beanutils-1.9.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stringtemplate-3.2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/gson-2.2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/okio-1.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/slf4j-api-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jsr305-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-el-1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-core-3.2.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xz-1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-client-2.6.0-cdh5.15.2.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"LargeBlocks","App ID":"application_1553914137147_0018","Timestamp":1554755984286,"User":"systest"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1554755994596,"Executor ID":"1","Executor Info":{"Host":"test-2.vpc.company.com","Total Cores":1,"Log Urls":{"stdout":"http://test-2.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000002/systest/stdout?start=-4096","stderr":"http://test-2.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000002/systest/stderr?start=-4096"},"Attributes":{"NM_HTTP_ADDRESS":"test-2.vpc.company.com:8042","USER":"systest","LOG_FILES":"stderr,stdout","NM_HTTP_PORT":"8042","CLUSTER_ID":"","NM_PORT":"8041","HTTP_SCHEME":"http://","NM_HOST":"test-2.vpc.company.com","CONTAINER_ID":"container_1553914137147_0018_01_000002"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"test-2.vpc.company.com","Port":43764},"Maximum Memory":3820172083,"Timestamp":1554755994649,"Maximum Onheap Memory":3820172083,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1554755995291,"Executor ID":"2","Executor Info":{"Host":"test-4.vpc.company.com","Total Cores":1,"Log Urls":{"stdout":"http://test-4.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000003/systest/stdout?start=-4096","stderr":"http://test-4.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000003/systest/stderr?start=-4096"},"Attributes":{"NM_HTTP_ADDRESS":"test-4.vpc.company.com:8042","USER":"systest","LOG_FILES":"stderr,stdout","NM_HTTP_PORT":"8042","CLUSTER_ID":"","NM_PORT":"8041","HTTP_SCHEME":"http://","NM_HOST":"test-4.vpc.company.com","CONTAINER_ID":"container_1553914137147_0018_01_000003"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"test-4.vpc.company.com","Port":33179},"Maximum Memory":3820172083,"Timestamp":1554755995347,"Maximum Onheap Memory":3820172083,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1554755995362,"Executor ID":"3","Executor Info":{"Host":"test-3.vpc.company.com","Total Cores":1,"Log Urls":{"stdout":"http://test-3.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000004/systest/stdout?start=-4096","stderr":"http://test-3.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000004/systest/stderr?start=-4096"},"Attributes":{"NM_HTTP_ADDRESS":"test-3.vpc.company.com:8042","USER":"systest","LOG_FILES":"stderr,stdout","NM_HTTP_PORT":"8042","CLUSTER_ID":"","NM_PORT":"8041","HTTP_SCHEME":"http://","NM_HOST":"test-3.vpc.company.com","CONTAINER_ID":"container_1553914137147_0018_01_000004"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"test-3.vpc.company.com","Port":37641},"Maximum Memory":3820172083,"Timestamp":1554755995519,"Maximum Onheap Memory":3820172083,"Maximum Offheap Memory":0} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1554755995741,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:122","Number of Tasks":2,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.LargeBlocks$.cacheTest(LargeBlocks.scala:122)\ncom.company.spark.LargeBlocks$.main(LargeBlocks.scala:49)\ncom.company.spark.LargeBlocks.main(LargeBlocks.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:860)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:178)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:201)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:88)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:939)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:948)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Accumulables":[]}],"Stage IDs":[0],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:122","Number of Tasks":2,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.LargeBlocks$.cacheTest(LargeBlocks.scala:122)\ncom.company.spark.LargeBlocks$.main(LargeBlocks.scala:49)\ncom.company.spark.LargeBlocks.main(LargeBlocks.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:860)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:178)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:201)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:88)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:939)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:948)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1554755995763,"Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1554755996103,"Executor ID":"3","Host":"test-3.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1554755996119,"Executor ID":"1","Host":"test-2.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1554755996103,"Executor ID":"3","Host":"test-3.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1554756012321,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":128,"Value":128,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":840,"Value":840,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":14868695279,"Value":14868695279,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":15006,"Value":15006,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1050068876,"Value":1050068876,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1152,"Value":1152,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":499823904,"JVMOffHeapMemory":58989648,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":5514,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":5514,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10440,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9680105472,"ProcessTreeJVMRSSMemory":776970240,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":19,"MinorGCTime":122,"MajorGCCount":2,"MajorGCTime":63},"Task Metrics":{"Executor Deserialize Time":1152,"Executor Deserialize CPU Time":1050068876,"Executor Run Time":15006,"Executor CPU Time":14868695279,"Result Size":840,"JVM GC Time":128,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1554755996119,"Executor ID":"1","Host":"test-2.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1554756012377,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":125,"Value":253,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":840,"Value":1680,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":14826135005,"Value":29694830284,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":14929,"Value":29935,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1127672479,"Value":2177741355,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1294,"Value":2446,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":508119376,"JVMOffHeapMemory":58916608,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":5514,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":5514,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10246,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9678606336,"ProcessTreeJVMRSSMemory":858959872,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":19,"MinorGCTime":118,"MajorGCCount":2,"MajorGCTime":60},"Task Metrics":{"Executor Deserialize Time":1294,"Executor Deserialize CPU Time":1127672479,"Executor Run Time":14929,"Executor CPU Time":14826135005,"Result Size":840,"JVM GC Time":125,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":140508272,"JVMOffHeapMemory":88453064,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":5514,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":5514,"OffHeapUnifiedMemory":0,"DirectPoolMemory":137861,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":8286560256,"ProcessTreeJVMRSSMemory":497471488,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":55,"MajorGCCount":3,"MajorGCTime":144}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":508119376,"JVMOffHeapMemory":58916608,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":5514,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":5514,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10246,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9678606336,"ProcessTreeJVMRSSMemory":858959872,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":19,"MinorGCTime":118,"MajorGCCount":2,"MajorGCTime":60}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":499823904,"JVMOffHeapMemory":58989648,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":5514,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":5514,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10440,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9680105472,"ProcessTreeJVMRSSMemory":776970240,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":19,"MinorGCTime":122,"MajorGCCount":2,"MajorGCTime":63}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:122","Number of Tasks":2,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.LargeBlocks$.cacheTest(LargeBlocks.scala:122)\ncom.company.spark.LargeBlocks$.main(LargeBlocks.scala:49)\ncom.company.spark.LargeBlocks.main(LargeBlocks.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:860)\norg.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:178)\norg.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:201)\norg.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:88)\norg.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:939)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:948)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)","Submission Time":1554755995763,"Completion Time":1554756012379,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":29935,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":253,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":1680,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2177741355,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":29694830284,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":2446,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1554756012383,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1554756012414,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:289","Number of Tasks":2,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"5\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:289","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.Verify$.verify(LargeBlocks.scala:289)\ncom.company.spark.LargeBlocks$$anon$2.run(LargeBlocks.scala:159)\njava.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[]}],"Stage IDs":[1],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:289","Number of Tasks":2,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"5\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:289","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.Verify$.verify(LargeBlocks.scala:289)\ncom.company.spark.LargeBlocks$$anon$2.run(LargeBlocks.scala:159)\njava.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1554756012416,"Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1554756012429,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:289","Number of Tasks":2,"RDD Info":[{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:289","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.Verify$.verify(LargeBlocks.scala:289)\ncom.company.spark.LargeBlocks$$anon$2.run(LargeBlocks.scala:159)\njava.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[]}],"Stage IDs":[2],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:289","Number of Tasks":2,"RDD Info":[{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:289","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.Verify$.verify(LargeBlocks.scala:289)\ncom.company.spark.LargeBlocks$$anon$2.run(LargeBlocks.scala:159)\njava.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1554756012430,"Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1554756012429,"Executor ID":"3","Host":"test-3.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Launch Time":1554756012430,"Executor ID":"1","Host":"test-2.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1554756016145,"Executor ID":"2","Host":"test-4.vpc.company.com","Locality":"RACK_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1554756026880,"Executor ID":"1","Host":"test-2.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Launch Time":1554756012430,"Executor ID":"1","Host":"test-2.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1554756026881,"Failed":false,"Killed":false,"Accumulables":[{"ID":47,"Name":"internal.metrics.input.recordsRead","Update":10000,"Value":10000,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.input.bytesRead","Update":2500050000,"Value":2500050000,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.jvmGCTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.resultSize","Update":924,"Value":924,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorCpuTime","Update":14192326943,"Value":14192326943,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorRunTime","Update":14413,"Value":14413,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.executorDeserializeCpuTime","Update":22769433,"Value":22769433,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"internal.metrics.executorDeserializeTime","Update":30,"Value":30,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":850753456,"JVMOffHeapMemory":60309664,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":12537,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":12537,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10658,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9678606336,"ProcessTreeJVMRSSMemory":1078870016,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":23,"MinorGCTime":129,"MajorGCCount":2,"MajorGCTime":60},"Task Metrics":{"Executor Deserialize Time":30,"Executor Deserialize CPU Time":22769433,"Executor Run Time":14413,"Executor CPU Time":14192326943,"Result Size":924,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":2500050000,"Records Read":10000},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1554756012429,"Executor ID":"3","Host":"test-3.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1554756027226,"Failed":false,"Killed":false,"Accumulables":[{"ID":47,"Name":"internal.metrics.input.recordsRead","Update":10000,"Value":20000,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.input.bytesRead","Update":2500050000,"Value":5000100000,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.jvmGCTime","Update":23,"Value":34,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.resultSize","Update":924,"Value":1848,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorCpuTime","Update":14505304892,"Value":28697631835,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorRunTime","Update":14738,"Value":29151,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.executorDeserializeCpuTime","Update":20320601,"Value":43090034,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"internal.metrics.executorDeserializeTime","Update":33,"Value":63,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":759479480,"JVMOffHeapMemory":61343616,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":12537,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":12537,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10865,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9691553792,"ProcessTreeJVMRSSMemory":964222976,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":24,"MinorGCTime":145,"MajorGCCount":2,"MajorGCTime":63},"Task Metrics":{"Executor Deserialize Time":33,"Executor Deserialize CPU Time":20320601,"Executor Run Time":14738,"Executor CPU Time":14505304892,"Result Size":924,"JVM GC Time":23,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":2500050000,"Records Read":10000},"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":186005832,"JVMOffHeapMemory":90852264,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":19560,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":19560,"OffHeapUnifiedMemory":0,"DirectPoolMemory":157182,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":8294973440,"ProcessTreeJVMRSSMemory":512442368,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":55,"MajorGCCount":3,"MajorGCTime":144}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":850753456,"JVMOffHeapMemory":60309664,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":12537,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":12537,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10658,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9678606336,"ProcessTreeJVMRSSMemory":1078870016,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":23,"MinorGCTime":129,"MajorGCCount":2,"MajorGCTime":60}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":759479480,"JVMOffHeapMemory":61343616,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":12537,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":12537,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10865,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9691553792,"ProcessTreeJVMRSSMemory":964222976,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":24,"MinorGCTime":145,"MajorGCCount":2,"MajorGCTime":63}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:289","Number of Tasks":2,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"5\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:289","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.Verify$.verify(LargeBlocks.scala:289)\ncom.company.spark.LargeBlocks$$anon$2.run(LargeBlocks.scala:159)\njava.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1554756012416,"Completion Time":1554756027227,"Accumulables":[{"ID":26,"Name":"internal.metrics.executorDeserializeCpuTime","Value":43090034,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.recordsRead","Value":20000,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.resultSize","Value":1848,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.input.bytesRead","Value":5000100000,"Internal":true,"Count Failed Values":true},{"ID":25,"Name":"internal.metrics.executorDeserializeTime","Value":63,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorCpuTime","Value":28697631835,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorRunTime","Value":29151,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.jvmGCTime","Value":34,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1554756027227,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":1,"Attempt":0,"Launch Time":1554756026880,"Executor ID":"1","Host":"test-2.vpc.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1554756041071,"Failed":false,"Killed":false,"Accumulables":[{"ID":72,"Name":"internal.metrics.input.recordsRead","Update":10000,"Value":10000,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.input.bytesRead","Update":2500050000,"Value":2500050000,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.jvmGCTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.resultSize","Update":924,"Value":924,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorCpuTime","Update":13858940111,"Value":13858940111,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorRunTime","Update":14080,"Value":14080,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9678662,"Value":9678662,"Internal":true,"Count Failed Values":true},{"ID":50,"Name":"internal.metrics.executorDeserializeTime","Update":104,"Value":104,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":984736560,"JVMOffHeapMemory":60829472,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":19560,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":19560,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10742,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9678606336,"ProcessTreeJVMRSSMemory":1210867712,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":27,"MinorGCTime":140,"MajorGCCount":2,"MajorGCTime":60},"Task Metrics":{"Executor Deserialize Time":104,"Executor Deserialize CPU Time":9678662,"Executor Run Time":14080,"Executor CPU Time":13858940111,"Result Size":924,"JVM GC Time":11,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":2500050000,"Records Read":10000},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1554756016145,"Executor ID":"2","Host":"test-4.vpc.company.com","Locality":"RACK_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1554756046447,"Failed":false,"Killed":false,"Accumulables":[{"ID":72,"Name":"internal.metrics.input.recordsRead","Update":10000,"Value":20000,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.input.bytesRead","Update":2500050000,"Value":5000100000,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.jvmGCTime","Update":121,"Value":132,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.resultSize","Update":924,"Value":1848,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorCpuTime","Update":12644117227,"Value":26503057338,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorRunTime","Update":29084,"Value":43164,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1067613469,"Value":1077292131,"Internal":true,"Count Failed Values":true},{"ID":50,"Name":"internal.metrics.executorDeserializeTime","Update":1194,"Value":1298,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":462135680,"JVMOffHeapMemory":62717520,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":7023,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":7023,"OffHeapUnifiedMemory":0,"DirectPoolMemory":12781,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9684373504,"ProcessTreeJVMRSSMemory":703639552,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":15,"MinorGCTime":106,"MajorGCCount":2,"MajorGCTime":75},"Task Metrics":{"Executor Deserialize Time":1194,"Executor Deserialize CPU Time":1067613469,"Executor Run Time":29084,"Executor CPU Time":12644117227,"Result Size":924,"JVM GC Time":121,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":2500050000,"Records Read":10000},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":462135680,"JVMOffHeapMemory":62717520,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":7023,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":7023,"OffHeapUnifiedMemory":0,"DirectPoolMemory":12781,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9684373504,"ProcessTreeJVMRSSMemory":703639552,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":15,"MinorGCTime":106,"MajorGCCount":2,"MajorGCTime":75}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":209883992,"JVMOffHeapMemory":91926448,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":19560,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":19560,"OffHeapUnifiedMemory":0,"DirectPoolMemory":157182,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":8296026112,"ProcessTreeJVMRSSMemory":526491648,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":55,"MajorGCCount":3,"MajorGCTime":144}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":984736560,"JVMOffHeapMemory":60829472,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":19560,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":19560,"OffHeapUnifiedMemory":0,"DirectPoolMemory":10742,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":9678606336,"ProcessTreeJVMRSSMemory":1210867712,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":27,"MinorGCTime":140,"MajorGCCount":2,"MajorGCTime":60}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at LargeBlocks.scala:289","Number of Tasks":2,"RDD Info":[{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:289","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitionsWithIndex\"}","Callsite":"mapPartitionsWithIndex at LargeBlocks.scala:214","Parent IDs":[0],"Storage Level":{"Use Disk":true,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at LargeBlocks.scala:214","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1184)\ncom.company.spark.Verify$.verify(LargeBlocks.scala:289)\ncom.company.spark.LargeBlocks$$anon$2.run(LargeBlocks.scala:159)\njava.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1554756012430,"Completion Time":1554756046447,"Accumulables":[{"ID":50,"Name":"internal.metrics.executorDeserializeTime","Value":1298,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.input.bytesRead","Value":5000100000,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorCpuTime","Value":26503057338,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.jvmGCTime","Value":132,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorRunTime","Value":43164,"Internal":true,"Count Failed Values":true},{"ID":72,"Name":"internal.metrics.input.recordsRead","Value":20000,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.resultSize","Value":1848,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1077292131,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1554756046448,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1554756046454} 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 c12b71a51876..05318416674b 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 @@ -133,11 +133,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "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", - "executor list with executor process tree metrics json" -> - "applications/application_1538416563558_0014/executors", - "executor list with executor garbage collection metrics json" -> - "applications/application_1536831636016_59384/1/executors", + "applications/application_1553914137147_0018/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/dev/.rat-excludes b/dev/.rat-excludes index 5a420eeec417..7cc3cf3c9938 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -82,9 +82,7 @@ app-20180109111548-0000 app-20161115172038-0000 app-20161116163331-0000 application_1516285256255_0012 -application_1506645932520_24630151 -application_1538416563558_0014 -application_1536831636016_59384_1 +application_1553914137147_0018 stat local-1422981759269 local-1422981780767 From e062e60afa94bc6d5e83e633d88321007202ce77 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 15 May 2019 16:23:13 -0700 Subject: [PATCH 16/22] [SPARK-26329][CORE] Get executor updates and reset the peaks in a single pass. Also, make driver stage key a constant; and use SAM syntax. --- .../main/scala/org/apache/spark/SparkContext.scala | 5 ++--- .../scala/org/apache/spark/executor/Executor.scala | 2 ++ .../spark/executor/ExecutorMetricsPoller.scala | 13 +++++-------- .../spark/scheduler/EventLoggingListener.scala | 5 +++-- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 8 ++++---- 5 files changed, 16 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 96b69d40e1d0..aa71b21caa30 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2428,9 +2428,8 @@ class SparkContext(config: SparkConf) extends Logging { private def reportHeartBeat(): Unit = { val currentMetrics = ExecutorMetrics.getCurrentMetrics(env.memoryManager) val driverUpdates = new HashMap[(Int, Int), ExecutorMetrics] - // In the driver, we do not track per-stage metrics, so use a dummy stage - // for the key - driverUpdates.put((-1, -1), new ExecutorMetrics(currentMetrics)) + // In the driver, we do not track per-stage metrics, so use a dummy stage for the key + driverUpdates.put(EventLoggingListener.DRIVER_STAGE_KEY, new ExecutorMetrics(currentMetrics)) val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0) listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates, driverUpdates)) 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 7e476da9a8b5..d22159a6bf26 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -590,6 +590,8 @@ private[spark] class Executor( logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}") val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) + // Here and below, put task metric peaks in a WrappedArray to expose them as a Seq + // without requiring a copy. val metricPeaks = WrappedArray.make(metricsPoller.getTaskMetricPeaks(taskId)) val serializedTK = ser.serialize(TaskKilled(t.reason, accUpdates, accums, metricPeaks)) execBackend.statusUpdate(taskId, TaskState.KILLED, serializedTK) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala index 5813d741d3b4..2cb3fc02dd50 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -97,9 +97,7 @@ private[spark] class ExecutorMetricsPoller( /** Starts the polling thread. */ def start(): Unit = { if (poller != null) { - val pollingTask = new Runnable() { - override def run(): Unit = Utils.logUncaughtExceptions(poll()) - } + val pollingTask: Runnable = () => Utils.logUncaughtExceptions(poll()) poller.scheduleAtFixedRate(pollingTask, 0L, pollingInterval, TimeUnit.MILLISECONDS) } } @@ -179,15 +177,14 @@ private[spark] class ExecutorMetricsPoller( * (the last time this method was called). */ def getExecutorUpdates(): HashMap[StageKey, ExecutorMetrics] = { - // build the executor level memory metrics val executorUpdates = new HashMap[StageKey, ExecutorMetrics] - stageTCMP.forEach((k, v) => executorUpdates.put(k, new ExecutorMetrics(v._2))) - // reset the peaks - def resetPeaks(k: StageKey, v: TCMP): TCMP = + def getUpdateAndResetPeaks(k: StageKey, v: TCMP): TCMP = { + executorUpdates.put(k, new ExecutorMetrics(v._2)) (v._1, new AtomicLongArray(ExecutorMetricType.numMetrics)) + } - stageTCMP.replaceAll(resetPeaks) + stageTCMP.replaceAll(getUpdateAndResetPeaks) executorUpdates } 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 0f5aeb18f517..b1603990efb9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -283,8 +283,7 @@ private[spark] class EventLoggingListener( // If the update came from the driver, stageKey1 will be the dummy key (-1, -1), // so record those peaks for all active stages. // Otherwise, record the peaks for the matching stage. - val driverStageKey = (-1, -1) - if (stageKey1 == driverStageKey || stageKey1 == stageKey2) { + if (stageKey1 == DRIVER_STAGE_KEY || stageKey1 == stageKey2) { val metrics = metricsPerExecutor.getOrElseUpdate( event.execId, new ExecutorMetrics()) metrics.compareAndUpdatePeakValues(peaks) @@ -349,6 +348,8 @@ private[spark] object EventLoggingListener extends Logging { // Suffix applied to the names of files still being written by applications. val IN_PROGRESS = ".inprogress" val DEFAULT_LOG_DIR = "/tmp/spark-events" + // Dummy stage key used by driver in executor metrics updates + val DRIVER_STAGE_KEY = (-1, -1) private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) 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 ed7e43a9ae86..cff3ebf2fb7e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -2883,7 +2883,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Finish the first task of the shuffle map stage. runEvent(makeCompletionEvent( taskSets(0).tasks(0), Success, makeMapStatus("hostA", 4), - Seq.empty, createFakeTaskInfoWithId(0))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(0))) // The second task of the shuffle map stage failed with FetchFailed. runEvent(makeCompletionEvent( @@ -2899,19 +2899,19 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Finish the first task of the second attempt of the shuffle map stage. runEvent(makeCompletionEvent( taskSets(1).tasks(0), Success, makeMapStatus("hostA", 4), - Seq.empty, createFakeTaskInfoWithId(0))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(0))) // Finish the third task of the first attempt of the shuffle map stage. runEvent(makeCompletionEvent( taskSets(0).tasks(2), Success, makeMapStatus("hostA", 4), - Seq.empty, createFakeTaskInfoWithId(0))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(0))) assert(tasksMarkedAsCompleted.length == 1) assert(tasksMarkedAsCompleted.head.partitionId == 2) // Finish the forth task of the first attempt of the shuffle map stage. runEvent(makeCompletionEvent( taskSets(0).tasks(3), Success, makeMapStatus("hostA", 4), - Seq.empty, createFakeTaskInfoWithId(0))) + Seq.empty, Array.empty, createFakeTaskInfoWithId(0))) assert(tasksMarkedAsCompleted.length == 2) assert(tasksMarkedAsCompleted.last.partitionId == 3) From 20b4b7e2efdfcadf4f8ace132087924539fbe3cb Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 3 Jul 2019 13:53:00 -0700 Subject: [PATCH 17/22] [SPARK-26329][CORE] Test fixes after rebase on master. --- .../apache/spark/ExecutorAllocationManagerSuite.scala | 2 +- .../scheduler/dynalloc/ExecutorMonitorSuite.scala | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 00a9e7fe21dc..07fb323cfc35 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -1032,7 +1032,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private def onExecutorIdle(manager: ExecutorAllocationManager, id: String): Unit = { val info = new TaskInfo(1, 1, 1, 0, id, "foo.example.com", TaskLocality.PROCESS_LOCAL, false) info.markFinished(TaskState.FINISHED, 1) - post(SparkListenerTaskEnd(1, 1, "foo", Success, info, null)) + post(SparkListenerTaskEnd(1, 1, "foo", Success, info, new ExecutorMetrics, null)) } private def removeExecutor(manager: ExecutorAllocationManager, executorId: String): Boolean = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index 6a25754fcbe5..0630622f26fa 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -25,6 +25,7 @@ import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{doAnswer, mock, when} import org.apache.spark._ +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ @@ -102,11 +103,13 @@ class ExecutorMonitorSuite extends SparkFunSuite { monitor.onTaskStart(SparkListenerTaskStart(i, 1, taskInfo("1", 1))) assert(!monitor.isExecutorIdle("1")) - monitor.onTaskEnd(SparkListenerTaskEnd(i, 1, "foo", Success, taskInfo("1", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(i, 1, "foo", Success, taskInfo("1", 1), + new ExecutorMetrics, null)) assert(!monitor.isExecutorIdle("1")) } - monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("1", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("1", 1), + new ExecutorMetrics, null)) assert(monitor.isExecutorIdle("1")) assert(monitor.timedOutExecutors(clock.getTimeMillis()).isEmpty) assert(monitor.timedOutExecutors(clock.getTimeMillis() + idleTimeoutMs + 1) === Seq("1")) @@ -256,7 +259,8 @@ class ExecutorMonitorSuite extends SparkFunSuite { monitor.executorsKilled(Seq("3")) assert(monitor.pendingRemovalCount === 2) - monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("2", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("2", 1), + new ExecutorMetrics, null)) assert(monitor.timedOutExecutors().isEmpty) clock.advance(idleDeadline) assert(monitor.timedOutExecutors().toSet === Set("2")) From b898ad2af8be976de36f08e74358580275500971 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 3 Jul 2019 18:31:51 -0700 Subject: [PATCH 18/22] [SPARK-26329][CORE] Adopt some suggestions from attilapiros. --- .../org/apache/spark/executor/Executor.scala | 10 +++------- .../executor/ExecutorMetricsPoller.scala | 19 ++++++++----------- 2 files changed, 11 insertions(+), 18 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 d22159a6bf26..c337d2438128 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -432,9 +432,7 @@ private[spark] class Executor( env.mapOutputTracker.asInstanceOf[MapOutputTrackerWorker].updateEpoch(task.epoch) } - val stageId = task.stageId - val stageAttemptId = task.stageAttemptId - metricsPoller.onTaskStart(taskId, stageId, stageAttemptId) + metricsPoller.onTaskStart(taskId, task.stageId, task.stageAttemptId) taskStarted = true // Run the actual task and measure its runtime. @@ -667,11 +665,9 @@ private[spark] class Executor( } finally { runningTasks.remove(taskId) if (taskStarted) { - // This means the task was successfully deserialized, the stageId and stageAttemptId + // This means the task was successfully deserialized, its stageId and stageAttemptId // are known, and metricsPoller.onTaskStart was called. - val stageId = task.stageId - val stageAttemptId = task.stageAttemptId - metricsPoller.onTaskCompletion(taskId, stageId, stageAttemptId) + metricsPoller.onTaskCompletion(taskId, task.stageId, task.stageAttemptId) } } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala index 2cb3fc02dd50..ade0df3c5b8a 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -60,9 +60,9 @@ private[spark] class ExecutorMetricsPoller( private val poller = if (pollingInterval > 0) { - ThreadUtils.newDaemonSingleThreadScheduledExecutor("executor-metrics-poller") + Some(ThreadUtils.newDaemonSingleThreadScheduledExecutor("executor-metrics-poller")) } else { - null + None } /** @@ -78,12 +78,9 @@ private[spark] class ExecutorMetricsPoller( // get the latest values for the metrics val latestMetrics = ExecutorMetrics.getCurrentMetrics(memoryManager) - def compareAndUpdate(current: Long, latest: Long): Long = - if (latest > current) latest else current - def updatePeaks(metrics: AtomicLongArray): Unit = { (0 until metrics.length).foreach { i => - metrics.getAndAccumulate(i, latestMetrics(i), compareAndUpdate) + metrics.getAndAccumulate(i, latestMetrics(i), math.max) } } @@ -96,9 +93,9 @@ private[spark] class ExecutorMetricsPoller( /** Starts the polling thread. */ def start(): Unit = { - if (poller != null) { + poller.foreach { exec => val pollingTask: Runnable = () => Utils.logUncaughtExceptions(poll()) - poller.scheduleAtFixedRate(pollingTask, 0L, pollingInterval, TimeUnit.MILLISECONDS) + exec.scheduleAtFixedRate(pollingTask, 0L, pollingInterval, TimeUnit.MILLISECONDS) } } @@ -191,9 +188,9 @@ private[spark] class ExecutorMetricsPoller( /** Stops the polling thread. */ def stop(): Unit = { - if (poller != null) { - poller.shutdown() - poller.awaitTermination(10, TimeUnit.SECONDS) + poller.foreach { exec => + exec.shutdown() + exec.awaitTermination(10, TimeUnit.SECONDS) } } } From fbb55bfa295a5c9326b460dbad729fa82d530113 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Thu, 18 Jul 2019 19:49:30 -0700 Subject: [PATCH 19/22] [SPARK-26329][CORE] Address feedback from Imran Rashid. Fix some bugs in ExecutorSuite as well. --- .../executor/ExecutorMetricsPoller.scala | 49 +++++++-------- .../scheduler/EventLoggingListener.scala | 4 +- .../apache/spark/executor/ExecutorSuite.scala | 62 +++++++++---------- .../apache/spark/util/JsonProtocolSuite.scala | 7 +-- 4 files changed, 56 insertions(+), 66 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala index ade0df3c5b8a..1c0b970de447 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -22,14 +22,12 @@ import java.util.concurrent.atomic.{AtomicLong, AtomicLongArray} import scala.collection.mutable.HashMap -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.memory.MemoryManager import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.util.{ThreadUtils, Utils} /** - * :: DeveloperApi :: * A class that polls executor metrics, and tracks their peaks per task and per stage. * Each executor keeps an instance of this class. * The poll method polls the executor metrics, and is either run in its own thread or @@ -38,19 +36,31 @@ import org.apache.spark.util.{ThreadUtils, Utils} * executor's task runner threads concurrently with the polling thread. One thread may * update one of these maps while another reads it, so the reading thread may not get * the latest metrics, but this is ok. + * One ConcurrentHashMap tracks the number of running tasks and the executor metric + * peaks for each stage. A positive task count means the stage is active. When the task + * count reaches zero for a stage, we remove the entry from the map. That way, the map + * only contains entries for active stages and does not grow without bound. On every + * heartbeat, the executor gets the per-stage metric peaks from this class and sends + * them and the peaks are reset. + * The other ConcurrentHashMap tracks the executor metric peaks for each task (the peaks + * seen while each task is running). At task end, these peaks are sent with the task + * result by the task runner. + * The reason we track executor metric peaks per task in addition to per stage is: + * If between heartbeats, a stage completes, so there are no more running tasks for that + * stage, then in the next heartbeat, there are no metrics sent for that stage; however, + * at the end of a task that belonged to that stage, the metrics would have been sent + * in the task result, so we do not lose those peaks. * * @param memoryManager the memory manager used by the executor. * @param pollingInterval the polling interval in milliseconds. */ -@DeveloperApi private[spark] class ExecutorMetricsPoller( memoryManager: MemoryManager, - pollingInterval: Long) - extends Logging { + pollingInterval: Long) extends Logging { type StageKey = (Int, Int) - // tuple for Task Count and Metric Peaks - type TCMP = (AtomicLong, AtomicLongArray) + // Task Count and Metric Peaks + case class TCMP(count: AtomicLong, peaks: AtomicLongArray) // Map of (stageId, stageAttemptId) to (count of running tasks, executor metric peaks) private val stageTCMP = new ConcurrentHashMap[StageKey, TCMP] @@ -85,7 +95,7 @@ private[spark] class ExecutorMetricsPoller( } // for each active stage, update the peaks - stageTCMP.forEachValue(LONG_MAX_VALUE, v => updatePeaks(v._2)) + stageTCMP.forEachValue(LONG_MAX_VALUE, v => updatePeaks(v.peaks)) // for each running task, update the peaks taskMetricPeaks.forEachValue(LONG_MAX_VALUE, updatePeaks) @@ -101,10 +111,6 @@ private[spark] class ExecutorMetricsPoller( /** * Called by TaskRunner#run. - * - * @param taskId the id of the task being run. - * @param stageId the id of the stage the task belongs to. - * @param stageAttemptId the attempt number of the stage the task belongs to. */ def onTaskStart(taskId: Long, stageId: Int, stageAttemptId: Int): Unit = { // Put an entry in taskMetricPeaks for the task. @@ -112,27 +118,22 @@ private[spark] class ExecutorMetricsPoller( // Put a new entry in stageTCMP for the stage if there isn't one already. // Increment the task count. - val (count, _) = stageTCMP.computeIfAbsent((stageId, stageAttemptId), - _ => (new AtomicLong(0), new AtomicLongArray(ExecutorMetricType.numMetrics))) - val stageCount = count.incrementAndGet() + val countAndPeaks = stageTCMP.computeIfAbsent((stageId, stageAttemptId), + _ => TCMP(new AtomicLong(0), new AtomicLongArray(ExecutorMetricType.numMetrics))) + val stageCount = countAndPeaks.count.incrementAndGet() logDebug(s"stageTCMP: ($stageId, $stageAttemptId) -> $stageCount") } /** * Called by TaskRunner#run. It should only be called if onTaskStart has been called with * the same arguments. - * - * @param taskId the id of the task that was run. - * @param stageId the id of the stage the task belongs to. - * @param stageAttemptId the attempt number of the stage the task belongs to. */ def onTaskCompletion(taskId: Long, stageId: Int, stageAttemptId: Int): Unit = { // Decrement the task count. // Remove the entry from stageTCMP if the task count reaches zero. def decrementCount(stage: StageKey, countAndPeaks: TCMP): TCMP = { - val count = countAndPeaks._1 - val countValue = count.decrementAndGet() + val countValue = countAndPeaks.count.decrementAndGet() if (countValue == 0L) { logDebug(s"removing (${stage._1}, ${stage._2}) from stageTCMP") null @@ -150,8 +151,6 @@ private[spark] class ExecutorMetricsPoller( /** * Called by TaskRunner#run. - * - * @param taskId the id of the task that was run. */ def getTaskMetricPeaks(taskId: Long): Array[Long] = { // If this is called with an invalid taskId or a valid taskId but the task was killed and @@ -177,8 +176,8 @@ private[spark] class ExecutorMetricsPoller( val executorUpdates = new HashMap[StageKey, ExecutorMetrics] def getUpdateAndResetPeaks(k: StageKey, v: TCMP): TCMP = { - executorUpdates.put(k, new ExecutorMetrics(v._2)) - (v._1, new AtomicLongArray(ExecutorMetricType.numMetrics)) + executorUpdates.put(k, new ExecutorMetrics(v.peaks)) + TCMP(v.count, new AtomicLongArray(ExecutorMetricType.numMetrics)) } stageTCMP.replaceAll(getUpdateAndResetPeaks) 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 b1603990efb9..48eb2da3015f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -278,7 +278,7 @@ private[spark] class EventLoggingListener( override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { if (shouldLogStageExecutorMetrics) { - event.executorUpdates.foreach { case (stageKey1, peaks) => + event.executorUpdates.foreach { case (stageKey1, newPeaks) => liveStageExecutorMetrics.foreach { case (stageKey2, metricsPerExecutor) => // If the update came from the driver, stageKey1 will be the dummy key (-1, -1), // so record those peaks for all active stages. @@ -286,7 +286,7 @@ private[spark] class EventLoggingListener( if (stageKey1 == DRIVER_STAGE_KEY || stageKey1 == stageKey2) { val metrics = metricsPerExecutor.getOrElseUpdate( event.execId, new ExecutorMetrics()) - metrics.compareAndUpdatePeakValues(peaks) + metrics.compareAndUpdatePeakValues(newPeaks) } } } diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index eb5220e8bfe5..ac7e4b51ebc2 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -39,6 +39,7 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.TaskState.TaskState +import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.memory.TestMemoryManager @@ -55,6 +56,12 @@ import org.apache.spark.util.{LongAccumulator, UninterruptibleThread} class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar with Eventually with PrivateMethodTester { + override def afterEach() { + // Unset any latches after each test; each test that needs them initializes new ones. + ExecutorSuiteHelper.latches = null + super.afterEach() + } + test("SPARK-15963: Catch `TaskKilledException` correctly in Executor.TaskRunner") { // mock some objects to make Executor.launchTask() happy val conf = new SparkConf @@ -136,10 +143,6 @@ class ExecutorSuite extends SparkFunSuite } } - // This test does not use ExecutorSuiteHelper.latches. - // It instantiates a FetchFailureHidingRDD with throwOOM = false and interrupt = false. - // It calls runTaskGetFailReasonAndExceptionHandler (via runTaskAndGetFailReason) with - // killTask = false and poll = false. test("SPARK-19276: Handle FetchFailedExceptions that are hidden by user exceptions") { val conf = new SparkConf().setMaster("local").setAppName("executor suite test") sc = new SparkContext(conf) @@ -151,7 +154,8 @@ class ExecutorSuite extends SparkFunSuite // fetch failure, not a generic exception from user code. val inputRDD = new FetchFailureThrowingRDD(sc) val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = false, interrupt = false) - val taskDescription = createResultTaskDescription(serializer, resultFunc, secondRDD, 1) + val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array()) + val taskDescription = createResultTaskDescription(serializer, taskBinary, secondRDD, 1) val failReason = runTaskAndGetFailReason(taskDescription) assert(failReason.isInstanceOf[FetchFailed]) @@ -211,14 +215,12 @@ class ExecutorSuite extends SparkFunSuite // (and to poll executor metrics if necessary) ExecutorSuiteHelper.latches = new ExecutorSuiteHelper val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = oom, interrupt = !oom) - val taskDescription = createResultTaskDescription(serializer, resultFunc, secondRDD, 1) + val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array()) + val taskDescription = createResultTaskDescription(serializer, taskBinary, secondRDD, 1) runTaskGetFailReasonAndExceptionHandler(taskDescription, killTask = !oom, poll) } - // This test does not use ExecutorSuiteHelper.latches. - // It calls runTaskGetFailReasonAndExceptionHandler (via runTaskAndGetFailReason) with - // killTask = false and poll = false. test("Gracefully handle error in task deserialization") { val conf = new SparkConf val serializer = new JavaSerializer(conf) @@ -330,29 +332,24 @@ class ExecutorSuite extends SparkFunSuite ExecutorSuiteHelper.latches = new ExecutorSuiteHelper val resultFunc = (context: TaskContext, itr: Iterator[Int]) => { - ExecutorSuiteHelper.latches.latch1.await(300, TimeUnit.MILLISECONDS) - ExecutorSuiteHelper.latches.latch2.countDown() - ExecutorSuiteHelper.latches.latch3.await(500, TimeUnit.MILLISECONDS) + // latch1 tells the test that the task is running, so it can ask the metricsPoller + // to poll; latch2 waits for the polling to be done + ExecutorSuiteHelper.latches.latch1.countDown() + ExecutorSuiteHelper.latches.latch2.await(5, TimeUnit.SECONDS) itr.size } val rdd = new RDD[Int](sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[Int] = { - val l = List(1) - l.iterator + Iterator(1) } override protected def getPartitions: Array[Partition] = { Array(new SimplePartition) } } - val taskDescription = createResultTaskDescription(serializer, resultFunc, rdd, 0) + val taskBinary = sc.broadcast(serializer.serialize((rdd, resultFunc)).array()) + val taskDescription = createResultTaskDescription(serializer, taskBinary, rdd, 0) val mockBackend = mock[ExecutorBackend] - when(mockBackend.statusUpdate(any(), meq(TaskState.RUNNING), any())) - .thenAnswer(new Answer[Unit] { - override def answer(invocationOnMock: InvocationOnMock): Unit = { - ExecutorSuiteHelper.latches.latch1.countDown() - } - }) var executor: Executor = null try { executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true) @@ -360,10 +357,10 @@ class ExecutorSuite extends SparkFunSuite // Ensure that the executor's metricsPoller is polled so that values are recorded for // the task metrics - ExecutorSuiteHelper.latches.latch2.await(500, TimeUnit.MILLISECONDS) + ExecutorSuiteHelper.latches.latch1.await(5, TimeUnit.SECONDS) executor.metricsPoller.poll() - ExecutorSuiteHelper.latches.latch3.countDown() - eventually(timeout(1.seconds), interval(10.milliseconds)) { + ExecutorSuiteHelper.latches.latch2.countDown() + eventually(timeout(5.seconds), interval(10.milliseconds)) { assert(executor.numRunningTasks === 0) } } finally { @@ -421,10 +418,9 @@ class ExecutorSuite extends SparkFunSuite private def createResultTaskDescription( serializer: SerializerInstance, - resultFunc: (TaskContext, Iterator[Int]) => Int, + taskBinary: Broadcast[Array[Byte]], rdd: RDD[Int], stageId: Int): TaskDescription = { - val taskBinary = sc.broadcast(serializer.serialize((rdd, resultFunc)).array()) val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array() val task = new ResultTask( stageId = stageId, @@ -459,8 +455,6 @@ class ExecutorSuite extends SparkFunSuite runTaskGetFailReasonAndExceptionHandler(taskDescription, false)._1 } - // NOTE: This method is ever only called with killTask = false and poll = false when we are - // not using ExecutorSuiteHelper.latches. private def runTaskGetFailReasonAndExceptionHandler( taskDescription: TaskDescription, killTask: Boolean, @@ -492,11 +486,11 @@ class ExecutorSuite extends SparkFunSuite } killingThread.start() } else { - // As noted above, when killTask = false and poll = false, we are not using latches; - // thus we only need to wait for latch1 and countdown latch2 when poll = true. - if (poll) { - ExecutorSuiteHelper.latches.latch1.await(1, TimeUnit.SECONDS) - executor.metricsPoller.poll() + if (ExecutorSuiteHelper.latches != null) { + ExecutorSuiteHelper.latches.latch1.await(5, TimeUnit.SECONDS) + if (poll) { + executor.metricsPoller.poll() + } ExecutorSuiteHelper.latches.latch2.countDown() } } @@ -566,7 +560,7 @@ class FetchFailureHidingRDD( if (throwOOM) { // Allow executor metrics to be polled (if necessary) before throwing the OOMError ExecutorSuiteHelper.latches.latch1.countDown() - ExecutorSuiteHelper.latches.latch2.await(500, TimeUnit.MILLISECONDS) + ExecutorSuiteHelper.latches.latch2.await(5, TimeUnit.SECONDS) // scalastyle:off throwerror throw new OutOfMemoryError("OOM while handling another exception") // scalastyle:on throwerror 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 43be0881d8b2..a093fa686f19 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -562,9 +562,6 @@ private[spark] object JsonProtocolSuite extends Assertions { * --------------------------------- */ private[spark] def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) { - def lexOrder(x: (Int, Int), y: (Int, Int)) = - x._1 < y._1 || x._2 < y._2 - (event1, event2) match { case (e1: SparkListenerStageSubmitted, e2: SparkListenerStageSubmitted) => assert(e1.properties === e2.properties) @@ -612,8 +609,8 @@ private[spark] object JsonProtocolSuite extends Assertions { assertSeqEquals[AccumulableInfo](updates1, updates2, (a, b) => a.equals(b)) }) assertSeqEquals[((Int, Int), ExecutorMetrics)]( - e1.executorUpdates.toSeq.sortWith((x, y) => lexOrder(x._1, y._1)), - e2.executorUpdates.toSeq.sortWith((x, y) => lexOrder(x._1, y._1)), + e1.executorUpdates.toSeq.sortBy(_._1), + e2.executorUpdates.toSeq.sortBy(_._1), (a, b) => { val (k1, v1) = a val (k2, v2) = b From 99addf15dd4e994d6f5b69c08a6e809588bf9a16 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Thu, 18 Jul 2019 20:06:29 -0700 Subject: [PATCH 20/22] [SPARK-26329][CORE] Make TCMP case class private. --- .../scala/org/apache/spark/executor/ExecutorMetricsPoller.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala index 1c0b970de447..bbb4ab925634 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -60,7 +60,7 @@ private[spark] class ExecutorMetricsPoller( type StageKey = (Int, Int) // Task Count and Metric Peaks - case class TCMP(count: AtomicLong, peaks: AtomicLongArray) + private case class TCMP(count: AtomicLong, peaks: AtomicLongArray) // Map of (stageId, stageAttemptId) to (count of running tasks, executor metric peaks) private val stageTCMP = new ConcurrentHashMap[StageKey, TCMP] From 7331b27a08ec7a7d1f775bd13acdc4a355352963 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Mon, 29 Jul 2019 15:28:10 -0700 Subject: [PATCH 21/22] [SPARK-26329][CORE] Fix a test post-rebase. --- .../dynalloc/ExecutorMonitorSuite.scala | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index 0630622f26fa..e9330f8c22ce 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -291,15 +291,18 @@ class ExecutorMonitorSuite extends SparkFunSuite { // First a failed task, to make sure it does not count. monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) - monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", TaskResultLost, taskInfo("1", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", TaskResultLost, taskInfo("1", 1), + new ExecutorMetrics, null)) assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) - monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), + new ExecutorMetrics, null)) assert(monitor.timedOutExecutors(idleDeadline).isEmpty) monitor.onTaskStart(SparkListenerTaskStart(3, 0, taskInfo("1", 1))) - monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("1", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("1", 1), + new ExecutorMetrics, null)) assert(monitor.timedOutExecutors(idleDeadline).isEmpty) // Finish the jobs, now the executor should be idle, but with the shuffle timeout, since the @@ -351,11 +354,13 @@ class ExecutorMonitorSuite extends SparkFunSuite { monitor.onJobStart(SparkListenerJobStart(2, clock.getTimeMillis(), Seq(stage3, stage4))) monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) - monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), + new ExecutorMetrics, null)) assert(monitor.timedOutExecutors(idleDeadline) === Seq("2")) monitor.onTaskStart(SparkListenerTaskStart(3, 0, taskInfo("2", 1))) - monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("2", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("2", 1), + new ExecutorMetrics, null)) assert(monitor.timedOutExecutors(idleDeadline).isEmpty) monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) @@ -385,7 +390,8 @@ class ExecutorMonitorSuite extends SparkFunSuite { clock.advance(1000L) monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) - monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), + new ExecutorMetrics, null)) monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) assert(monitor.timedOutExecutors(idleDeadline).isEmpty) From 7556d6ab8615e4fd216da16ec02a4d0020f65ce3 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 31 Jul 2019 10:48:36 -0700 Subject: [PATCH 22/22] [SPARK-26329][CORE] Update a doc comment based on feedback from Imran Rashid. --- .../executor/ExecutorMetricsPoller.scala | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala index bbb4ab925634..805b0f729b12 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetricsPoller.scala @@ -36,20 +36,12 @@ import org.apache.spark.util.{ThreadUtils, Utils} * executor's task runner threads concurrently with the polling thread. One thread may * update one of these maps while another reads it, so the reading thread may not get * the latest metrics, but this is ok. - * One ConcurrentHashMap tracks the number of running tasks and the executor metric - * peaks for each stage. A positive task count means the stage is active. When the task - * count reaches zero for a stage, we remove the entry from the map. That way, the map - * only contains entries for active stages and does not grow without bound. On every - * heartbeat, the executor gets the per-stage metric peaks from this class and sends - * them and the peaks are reset. - * The other ConcurrentHashMap tracks the executor metric peaks for each task (the peaks - * seen while each task is running). At task end, these peaks are sent with the task - * result by the task runner. - * The reason we track executor metric peaks per task in addition to per stage is: - * If between heartbeats, a stage completes, so there are no more running tasks for that - * stage, then in the next heartbeat, there are no metrics sent for that stage; however, - * at the end of a task that belonged to that stage, the metrics would have been sent - * in the task result, so we do not lose those peaks. + * We track executor metric peaks per stage, as well as per task. The per-stage peaks + * are sent in executor heartbeats. That way, we get incremental updates of the metrics + * as the tasks are running, and if the executor dies we still have some metrics. The + * per-task peaks are sent in the task result at task end. These are useful for short + * tasks. If there are no heartbeats during the task, we still get the metrics polled + * for the task. * * @param memoryManager the memory manager used by the executor. * @param pollingInterval the polling interval in milliseconds.