diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 84091eef0430..1012755e068d 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -61,10 +61,17 @@ private[spark] class Heartbeater( /** * Get the current executor level metrics. These are returned as an array, with the index - * determined by ExecutorMetricType.values + * determined by ExecutorMetricType.metricToOffset */ def getCurrentMetrics(): ExecutorMetrics = { - val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray + + 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/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 1befd27de1cb..f19ac813fde3 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -27,17 +27,15 @@ import org.apache.spark.metrics.ExecutorMetricType */ @DeveloperApi class ExecutorMetrics private[spark] extends Serializable { - - // Metrics are indexed by ExecutorMetricType.values - private val metrics = new Array[Long](ExecutorMetricType.values.length) - + // Metrics are indexed by ExecutorMetricType.metricToOffset + private val metrics = new Array[Long](ExecutorMetricType.numMetrics) // the first element is initialized to -1, indicating that the values for the array // haven't been set yet. metrics(0) = -1 - /** Returns the value for the specified metricType. */ - def getMetricValue(metricType: ExecutorMetricType): Long = { - metrics(ExecutorMetricType.metricIdxMap(metricType)) + /** Returns the value for the specified metric. */ + def getMetricValue(metricName: String): Long = { + metrics(ExecutorMetricType.metricToOffset(metricName)) } /** Returns true if the values for the metrics have been set, false otherwise. */ @@ -49,14 +47,14 @@ class ExecutorMetrics private[spark] extends Serializable { } /** - * Constructor: create the ExecutorMetrics with the values specified. + * Constructor: create the ExecutorMetrics with using a given map. * * @param executorMetrics map of executor metric name to value */ private[spark] def this(executorMetrics: Map[String, Long]) { this() - (0 until ExecutorMetricType.values.length).foreach { idx => - metrics(idx) = executorMetrics.getOrElse(ExecutorMetricType.values(idx).name, 0L) + ExecutorMetricType.metricToOffset.foreach { case(name, idx) => + metrics(idx) = executorMetrics.getOrElse(name, 0L) } } @@ -69,9 +67,8 @@ class ExecutorMetrics private[spark] extends Serializable { */ private[spark] def compareAndUpdatePeakValues(executorMetrics: ExecutorMetrics): Boolean = { var updated = false - - (0 until ExecutorMetricType.values.length).foreach { idx => - if (executorMetrics.metrics(idx) > metrics(idx)) { + (0 until ExecutorMetricType.numMetrics).foreach { idx => + if (executorMetrics.metrics(idx) > metrics(idx)) { updated = true metrics(idx) = executorMetrics.metrics(idx) } diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala new file mode 100644 index 000000000000..af67f41e94af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import java.io._ +import java.nio.charset.Charset +import java.nio.file.{Files, Paths} +import java.util.Locale + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.Try + +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.util.Utils + + +private[spark] case class ProcfsMetrics( + jvmVmemTotal: Long, + jvmRSSTotal: Long, + pythonVmemTotal: Long, + pythonRSSTotal: Long, + otherVmemTotal: Long, + otherRSSTotal: Long) + +// Some of the ideas here are taken from the ProcfsBasedProcessTree class in hadoop +// project. +private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends Logging { + private val procfsStatFile = "stat" + private val testing = sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + private val pageSize = computePageSize() + private var isAvailable: Boolean = isProcfsAvailable + private val pid = computePid() + + private lazy val isProcfsAvailable: Boolean = { + if (testing) { + true + } + else { + val procDirExists = Try(Files.exists(Paths.get(procfsDir))).recover { + case ioe: IOException => + logWarning("Exception checking for procfs dir", ioe) + false + } + val shouldLogStageExecutorMetrics = + SparkEnv.get.conf.get(config.EVENT_LOG_STAGE_EXECUTOR_METRICS) + val shouldLogStageExecutorProcessTreeMetrics = + SparkEnv.get.conf.get(config.EVENT_LOG_PROCESS_TREE_METRICS) + procDirExists.get && shouldLogStageExecutorProcessTreeMetrics && shouldLogStageExecutorMetrics + } + } + + private def computePid(): Int = { + if (!isAvailable || testing) { + return -1; + } + try { + // This can be simplified in java9: + // https://docs.oracle.com/javase/9/docs/api/java/lang/ProcessHandle.html + val cmd = Array("bash", "-c", "echo $PPID") + val out = Utils.executeAndGetOutput(cmd) + Integer.parseInt(out.split("\n")(0)) + } + catch { + case e: SparkException => + logWarning("Exception when trying to compute process tree." + + " As a result reporting of ProcessTree metrics is stopped", e) + isAvailable = false + -1 + } + } + + private def computePageSize(): Long = { + if (testing) { + return 4096; + } + try { + val cmd = Array("getconf", "PAGESIZE") + val out = Utils.executeAndGetOutput(cmd) + Integer.parseInt(out.split("\n")(0)) + } catch { + case e: Exception => + logWarning("Exception when trying to compute pagesize, as a" + + " result reporting of ProcessTree metrics is stopped") + isAvailable = false + 0 + } + } + + private def computeProcessTree(): Set[Int] = { + if (!isAvailable || testing) { + return Set() + } + var ptree: Set[Int] = Set() + ptree += pid + val queue = mutable.Queue.empty[Int] + queue += pid + while ( !queue.isEmpty ) { + val p = queue.dequeue() + val c = getChildPids(p) + if (!c.isEmpty) { + queue ++= c + ptree ++= c.toSet + } + } + ptree + } + + private def getChildPids(pid: Int): ArrayBuffer[Int] = { + try { + val builder = new ProcessBuilder("pgrep", "-P", pid.toString) + val process = builder.start() + val childPidsInInt = mutable.ArrayBuffer.empty[Int] + def appendChildPid(s: String): Unit = { + if (s != "") { + logTrace("Found a child pid:" + s) + childPidsInInt += Integer.parseInt(s) + } + } + val stdoutThread = Utils.processStreamByLine("read stdout for pgrep", + process.getInputStream, appendChildPid) + val errorStringBuilder = new StringBuilder() + val stdErrThread = Utils.processStreamByLine( + "stderr for pgrep", + process.getErrorStream, + line => errorStringBuilder.append(line)) + val exitCode = process.waitFor() + stdoutThread.join() + stdErrThread.join() + val errorString = errorStringBuilder.toString() + // pgrep will have exit code of 1 if there are more than one child process + // and it will have a exit code of 2 if there is no child process + if (exitCode != 0 && exitCode > 2) { + val cmd = builder.command().toArray.mkString(" ") + logWarning(s"Process $cmd exited with code $exitCode and stderr: $errorString") + throw new SparkException(s"Process $cmd exited with code $exitCode") + } + childPidsInInt + } catch { + case e: Exception => + logWarning("Exception when trying to compute process tree." + + " As a result reporting of ProcessTree metrics is stopped.", e) + isAvailable = false + mutable.ArrayBuffer.empty[Int] + } + } + + def addProcfsMetricsFromOneProcess( + allMetrics: ProcfsMetrics, + pid: Int): ProcfsMetrics = { + + // The computation of RSS and Vmem are based on proc(5): + // http://man7.org/linux/man-pages/man5/proc.5.html + try { + val pidDir = new File(procfsDir, pid.toString) + def openReader(): BufferedReader = { + val f = new File(new File(procfsDir, pid.toString), procfsStatFile) + new BufferedReader(new InputStreamReader(new FileInputStream(f), Charset.forName("UTF-8"))) + } + Utils.tryWithResource(openReader) { in => + val procInfo = in.readLine + val procInfoSplit = procInfo.split(" ") + val vmem = procInfoSplit(22).toLong + val rssMem = procInfoSplit(23).toLong * pageSize + if (procInfoSplit(1).toLowerCase(Locale.US).contains("java")) { + allMetrics.copy( + jvmVmemTotal = allMetrics.jvmVmemTotal + vmem, + jvmRSSTotal = allMetrics.jvmRSSTotal + (rssMem) + ) + } + else if (procInfoSplit(1).toLowerCase(Locale.US).contains("python")) { + allMetrics.copy( + pythonVmemTotal = allMetrics.pythonVmemTotal + vmem, + pythonRSSTotal = allMetrics.pythonRSSTotal + (rssMem) + ) + } + else { + allMetrics.copy( + otherVmemTotal = allMetrics.otherVmemTotal + vmem, + otherRSSTotal = allMetrics.otherRSSTotal + (rssMem) + ) + } + } + } catch { + case f: IOException => + logWarning("There was a problem with reading" + + " the stat file of the process. ", f) + ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + } + + private[spark] def computeAllMetrics(): ProcfsMetrics = { + if (!isAvailable) { + return ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + val pids = computeProcessTree + var allMetrics = ProcfsMetrics(0, 0, 0, 0, 0, 0) + for (p <- pids) { + allMetrics = addProcfsMetricsFromOneProcess(allMetrics, p) + // if we had an error getting any of the metrics, we don't want to report partial metrics, as + // that would be misleading. + if (!isAvailable) { + return ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + } + allMetrics + } +} + +private[spark] object ProcfsMetricsGetter { + final val pTreeInfo = new ProcfsMetricsGetter +} 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 c8993e17bba6..14a22df87bfa 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 @@ -80,6 +80,11 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val EVENT_LOG_PROCESS_TREE_METRICS = + ConfigBuilder("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled") + .booleanConf + .createWithDefault(false) + private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index cd10dad25e87..704b36d3118b 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -19,25 +19,43 @@ package org.apache.spark.metrics import java.lang.management.{BufferPoolMXBean, ManagementFactory} import javax.management.ObjectName +import scala.collection.mutable + +import org.apache.spark.executor.ProcfsMetricsGetter import org.apache.spark.memory.MemoryManager /** * Executor metric types for executor-level metrics stored in ExecutorMetrics. */ sealed trait ExecutorMetricType { + private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] + private[spark] def names: Seq[String] +} + +sealed trait SingleValueExecutorMetricType extends ExecutorMetricType { + override private[spark] def names = { + Seq(getClass().getName(). + stripSuffix("$").split("""\.""").last) + } + + override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = { + val metrics = new Array[Long](1) + metrics(0) = getMetricValue(memoryManager) + metrics + } + private[spark] def getMetricValue(memoryManager: MemoryManager): Long - private[spark] val name = getClass().getName().stripSuffix("$").split("""\.""").last } private[spark] abstract class MemoryManagerExecutorMetricType( - f: MemoryManager => Long) extends ExecutorMetricType { + f: MemoryManager => Long) extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { f(memoryManager) } } private[spark] abstract class MBeanExecutorMetricType(mBeanName: String) - extends ExecutorMetricType { + extends SingleValueExecutorMetricType { private val bean = ManagementFactory.newPlatformMXBeanProxy( ManagementFactory.getPlatformMBeanServer, new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean]) @@ -47,18 +65,40 @@ private[spark] abstract class MBeanExecutorMetricType(mBeanName: String) } } -case object JVMHeapMemory extends ExecutorMetricType { +case object JVMHeapMemory extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed() } } -case object JVMOffHeapMemory extends ExecutorMetricType { +case object JVMOffHeapMemory extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed() } } +case object ProcessTreeMetrics extends ExecutorMetricType { + override val names = Seq( + "ProcessTreeJVMVMemory", + "ProcessTreeJVMRSSMemory", + "ProcessTreePythonVMemory", + "ProcessTreePythonRSSMemory", + "ProcessTreeOtherVMemory", + "ProcessTreeOtherRSSMemory") + + override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = { + val allMetrics = ProcfsMetricsGetter.pTreeInfo.computeAllMetrics() + val processTreeMetrics = new Array[Long](names.length) + processTreeMetrics(0) = allMetrics.jvmVmemTotal + processTreeMetrics(1) = allMetrics.jvmRSSTotal + processTreeMetrics(2) = allMetrics.pythonVmemTotal + processTreeMetrics(3) = allMetrics.pythonRSSTotal + processTreeMetrics(4) = allMetrics.otherVmemTotal + processTreeMetrics(5) = allMetrics.otherRSSTotal + processTreeMetrics + } +} + case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType( _.onHeapExecutionMemoryUsed) @@ -84,8 +124,9 @@ case object MappedPoolMemory extends MBeanExecutorMetricType( "java.nio:type=BufferPool,name=mapped") private[spark] object ExecutorMetricType { - // List of all executor metric types - val values = IndexedSeq( + + // List of all executor metric getters + val metricGetters = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, OnHeapExecutionMemory, @@ -95,10 +136,21 @@ private[spark] object ExecutorMetricType { OnHeapUnifiedMemory, OffHeapUnifiedMemory, DirectPoolMemory, - MappedPoolMemory + MappedPoolMemory, + ProcessTreeMetrics ) - // Map of executor metric type to its index in values. - val metricIdxMap = - Map[ExecutorMetricType, Int](ExecutorMetricType.values.zipWithIndex: _*) + + val (metricToOffset, numMetrics) = { + var numberOfMetrics = 0 + val definedMetricsAndOffset = mutable.LinkedHashMap.empty[String, Int] + metricGetters.foreach { m => + var metricInSet = 0 + (0 until m.names.length).foreach { idx => + definedMetricsAndOffset += (m.names(idx) -> (idx + numberOfMetrics)) + } + numberOfMetrics += m.names.length + } + (definedMetricsAndOffset, numberOfMetrics) + } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 30afd8b76972..d519ffd64f00 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -133,9 +133,9 @@ private[spark] class ExecutorMetricsJsonSerializer jsonGenerator: JsonGenerator, serializerProvider: SerializerProvider): Unit = { metrics.foreach { m: ExecutorMetrics => - val metricsMap = ExecutorMetricType.values.map { metricType => - metricType.name -> m.getMetricValue(metricType) - }.toMap + val metricsMap = ExecutorMetricType.metricToOffset.map { case (metric, _) => + metric -> m.getMetricValue(metric) + } jsonGenerator.writeObject(metricsMap) } } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 0cd8612b8fd1..348291fe5e7a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -394,10 +394,10 @@ private[spark] object JsonProtocol { /** Convert executor metrics to JSON. */ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { - val metrics = ExecutorMetricType.values.map{ metricType => - JField(metricType.name, executorMetrics.getMetricValue(metricType)) - } - JObject(metrics: _*) + val metrics = ExecutorMetricType.metricToOffset.map { case (m, _) => + JField(m, executorMetrics.getMetricValue(m)) + } + JObject(metrics.toSeq: _*) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -611,10 +611,10 @@ private[spark] object JsonProtocol { /** Extract the executor metrics from JSON. */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val metrics = - ExecutorMetricType.values.map { metric => - metric.name -> jsonOption(json \ metric.name).map(_.extract[Long]).getOrElse(0L) - }.toMap - new ExecutorMetrics(metrics) + ExecutorMetricType.metricToOffset.map { case (metric, _) => + metric -> jsonOption(json \ metric).map(_.extract[Long]).getOrElse(0L) + } + new ExecutorMetrics(metrics.toMap) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index eea6f595efd2..0f0ccf9858a3 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_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", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index 7bc7f31be097..e136a35a1e3a 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_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", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json index 9bf2086cc8e7..75674778dd1f 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 @@ -37,7 +37,13 @@ "DirectPoolMemory" : 397602, "MappedPoolMemory" : 0, "JVMHeapMemory" : 629553808, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } }, { "id" : "7", @@ -177,7 +183,13 @@ "DirectPoolMemory" : 126261, "MappedPoolMemory" : 0, "JVMHeapMemory" : 518613056, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } }, { "id" : "3", @@ -221,7 +233,13 @@ "DirectPoolMemory" : 87796, "MappedPoolMemory" : 0, "JVMHeapMemory" : 726805712, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } }, { "id" : "2", @@ -265,7 +283,13 @@ "DirectPoolMemory" : 87796, "MappedPoolMemory" : 0, "JVMHeapMemory" : 595946552, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } }, { "id" : "1", @@ -309,6 +333,12 @@ "DirectPoolMemory" : 98230, "MappedPoolMemory" : 0, "JVMHeapMemory" : 755008624, - "OffHeapStorageMemory" : 0 + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0 } } ] 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 new file mode 100644 index 000000000000..69efefe736dd --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json @@ -0,0 +1,98 @@ +[ { + "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 + } +}, { + "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 + } +} ] 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 9e1e65a35881..0ef9377dcb08 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_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", "attempts" : [ { @@ -28,19 +43,4 @@ "startTimeEpoch" : 1516300235119, "endTimeEpoch" : 1516300707938 } ] -}, { - "id" : "app-20180109111548-0000", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-01-09T10:15:42.372GMT", - "endTime" : "2018-01-09T10:24:37.606GMT", - "lastUpdated" : "", - "duration" : 535234, - "sparkUser" : "attilapiros", - "completed" : true, - "appSparkVersion" : "2.3.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "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 28c6bf1b3e01..ea9dc1b97afc 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_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", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json index f547b79f47e1..2a77071a9ffd 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_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", "attempts" : [ { diff --git a/core/src/test/resources/ProcfsMetrics/22763/stat b/core/src/test/resources/ProcfsMetrics/22763/stat new file mode 100644 index 000000000000..cea4b713d0ee --- /dev/null +++ b/core/src/test/resources/ProcfsMetrics/22763/stat @@ -0,0 +1 @@ +22763 (python2.7) S 22756 22756 7051 0 -1 1077944384 449 0 0 0 4 3 0 0 20 0 3 0 117445 360595456 1912 18446744073709551615 4194304 4196756 140726192435536 140726192432528 140707465485051 0 0 16781312 2 18446744073709551615 0 0 17 1 0 0 0 0 0 6294976 6295604 38744064 140726192440006 140726192440119 140726192440119 140726192443369 0 \ No newline at end of file diff --git a/core/src/test/resources/ProcfsMetrics/26109/stat b/core/src/test/resources/ProcfsMetrics/26109/stat new file mode 100644 index 000000000000..ae46bfabd047 --- /dev/null +++ b/core/src/test/resources/ProcfsMetrics/26109/stat @@ -0,0 +1 @@ +26109 (java) S 1 26107 5788 0 -1 1077944320 75354 0 0 0 572 52 0 0 20 0 34 0 4355257 4769947648 64114 18446744073709551615 4194304 4196468 140737190381776 140737190364320 139976994791319 0 0 0 16800975 18446744073709551615 0 0 17 2 0 0 0 0 0 6293624 6294260 11276288 140737190385424 140737190414250 140737190414250 140737190416335 0 diff --git a/core/src/test/resources/spark-events/application_1538416563558_0014 b/core/src/test/resources/spark-events/application_1538416563558_0014 new file mode 100644 index 000000000000..000288dbc454 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1538416563558_0014 @@ -0,0 +1,190 @@ +{"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/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 11a2db81f7c6..0ca285cad90c 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 @@ -83,6 +83,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers .set("spark.testing", "true") .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) .set("spark.eventLog.logStageExecutorMetrics.enabled", "true") + .set("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled", "true") conf.setAll(extraConf) provider = new FsHistoryProvider(conf) provider.checkForLogs() @@ -131,6 +132,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "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", "stage list json" -> "applications/local-1422981780767/stages", "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", diff --git a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala new file mode 100644 index 000000000000..9ed1497db5e1 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.SparkFunSuite + + +class ProcfsMetricsGetterSuite extends SparkFunSuite { + + val p = new ProcfsMetricsGetter(getTestResourcePath("ProcfsMetrics")) + + test("testGetProcessInfo") { + var r = ProcfsMetrics(0, 0, 0, 0, 0, 0) + r = p.addProcfsMetricsFromOneProcess(r, 26109) + assert(r.jvmVmemTotal == 4769947648L) + assert(r.jvmRSSTotal == 262610944) + assert(r.pythonVmemTotal == 0) + assert(r.pythonRSSTotal == 0) + + r = p.addProcfsMetricsFromOneProcess(r, 22763) + assert(r.pythonVmemTotal == 360595456) + assert(r.pythonRSSTotal == 7831552) + assert(r.jvmVmemTotal == 4769947648L) + assert(r.jvmRSSTotal == 262610944) + } +} 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 cecd6996df7b..0c04a93646d7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -282,53 +282,67 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), + new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), - // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 + new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))), + // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), + new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), + new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L))), // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), + new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), + new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9; + new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, + 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 3, 6, 7, 9; // initialize stage 1 peaks createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, + 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L))), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), - // enew ExecutorMetrics(xec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 + new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))), + // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), + new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, + 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 5000L, 2000L))), // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), + new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, + 2000L, 1500L, 1000L, 500L))), // exec 2: new stage 1 peak for metrics at index: 7 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), + new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, + 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L))), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), + new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, + 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L))), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), + new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, + 4000L, 6000L, 3000L, 5000L, 2000L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -342,20 +356,23 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // expected StageExecutorMetrics, for the given stage id and executor id val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] = - Map( - ((0, "1"), - new SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), - ((0, "2"), - new SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), - ((1, "1"), - new SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), - ((1, "2"), - new SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) - + 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)))), + ((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)))), + ((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)))), + ((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))))) // 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. @@ -456,9 +473,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(executorMetrics.execId === expectedMetrics.execId) assert(executorMetrics.stageId === expectedMetrics.stageId) assert(executorMetrics.stageAttemptId === expectedMetrics.stageAttemptId) - ExecutorMetricType.values.foreach { metricType => - assert(executorMetrics.executorMetrics.getMetricValue(metricType) === - expectedMetrics.executorMetrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(executorMetrics.executorMetrics.getMetricValue(metric._1) === + expectedMetrics.executorMetrics.getMetricValue(metric._1)) } case None => assert(false) 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 0b2bbd2fa8a7..b9ecb0050ccb 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1304,58 +1304,74 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))) // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L))) // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))) // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks listener.onStageSubmitted(createStageSubmittedEvent(1)) // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))) // complete stage 0, and 3 more updates for each executor with just // stage 1 running listener.onStageCompleted(createStageCompletedEvent(0)) // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))) // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L, 8000L, 5000L, + 7000L, 4000L, 6000L, 3000L))) // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, 2000L, + 1500L, 1000L, 500L))) // exec 2: new stage 1 peak for metrics at index: 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L, 7000L, 3000L, + 6000L, 2000L, 5000L, 1000L))) // exec 1: no new stage 1 peaks listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L, 4000L, 2500L, + 3000L, 1500, 2000L, 500L))) listener.onExecutorRemoved(createExecutorRemovedEvent(1)) // exec 2: new stage 1 peak for metrics at index: 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, 4000L, 6000L, + 3000L, 5000L, 2000L))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L, 9000L, 5000L, 8000L, 4000L, 7000L, 3000L))) // check that the stored peak values match the expected values expectedValues.foreach { case (id, metrics) => @@ -1363,8 +1379,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.values.foreach { metricType => - assert(actual.getMetricValue(metricType) === metrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) } case _ => assert(false) @@ -1383,23 +1399,29 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onStageSubmitted(createStageSubmittedEvent(0)) listener.onStageSubmitted(createStageSubmittedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))) + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L, 9000L, + 4000L, 8000L, 3000L, 7000L, 2000L)))) listener.onStageCompleted(createStageCompletedEvent(0)) // executor 1 is removed before stage 1 has finished, the stage executor metrics // are logged afterwards and should still be used to update the executor metrics. listener.onExecutorRemoved(createExecutorRemovedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L, 8000L, 5000L, + 7000L, 4000L, 6000L, 3000L)))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L, 9000L, 5000L, 8000L, 4000L, 7000L, 3000L))) // check that the stored peak values match the expected values for ((id, metrics) <- expectedValues) { @@ -1407,8 +1429,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.values.foreach { metricType => - assert(actual.getMetricValue(metricType) === metrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) } case _ => assert(false) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 1e0d2af9a471..303ca7cb8801 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -96,7 +96,8 @@ class JsonProtocolSuite extends SparkFunSuite { .accumulators().map(AccumulatorSuite.makeInfo) .zipWithIndex.map { case (a, i) => a.copy(id = i) } val executorUpdates = new ExecutorMetrics( - Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L)) + Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), Some(executorUpdates)) } @@ -105,8 +106,8 @@ class JsonProtocolSuite extends SparkFunSuite { "In your multitude...", 300), RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100L, 0L)) val stageExecutorMetrics = SparkListenerStageExecutorMetrics("1", 2, 3, - new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, 321L, 654L, 765L))) - + new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) testEvent(taskStart, taskStartJsonString) @@ -440,14 +441,14 @@ class JsonProtocolSuite extends SparkFunSuite { test("executorMetricsFromJson backward compatibility: handle missing metrics") { // any missing metrics should be set to 0 - val executorMetrics = new ExecutorMetrics( - Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 789L)) + val executorMetrics = new ExecutorMetrics(Array(12L, 23L, 45L, 67L, 78L, 89L, + 90L, 123L, 456L, 789L, 40L, 20L, 20L, 10L, 20L, 10L)) val oldExecutorMetricsJson = JsonProtocol.executorMetricsToJson(executorMetrics) .removeField( _._1 == "MappedPoolMemory") - val expectedExecutorMetrics = new ExecutorMetrics( - Array(12L, 23L, 45L, 67L, 78L, 89L, 90L, 123L, 456L, 0L)) - assertEquals(expectedExecutorMetrics, + val exepectedExecutorMetrics = new ExecutorMetrics(Array(12L, 23L, 45L, 67L, + 78L, 89L, 90L, 123L, 456L, 0L, 40L, 20L, 20L, 10L, 20L, 10L)) + assertEquals(exepectedExecutorMetrics, JsonProtocol.executorMetricsFromJson(oldExecutorMetricsJson)) } @@ -753,9 +754,9 @@ private[spark] object JsonProtocolSuite extends Assertions { assertStackTraceElementEquals) } - private def assertEquals(metrics1: ExecutorMetrics, metrics2: ExecutorMetrics) { - ExecutorMetricType.values.foreach { metricType => - assert(metrics1.getMetricValue(metricType) === metrics2.getMetricValue(metricType)) + private def assertEquals(metrics1: ExecutorMetrics, metrics2: ExecutorMetrics): Unit = { + ExecutorMetricType.metricToOffset.foreach { metric => + assert(metrics1.getMetricValue(metric._1) === metrics2.getMetricValue(metric._1)) } } @@ -872,13 +873,14 @@ private[spark] object JsonProtocolSuite extends Assertions { if (includeTaskMetrics) { Seq((1L, 1, 1, Seq(makeAccumulableInfo(1, false, false, None), makeAccumulableInfo(2, false, false, None)))) - } else { + } else { Seq() } val executorMetricsUpdate = if (includeExecutorMetrics) { - Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L))) - } else { + Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, + 0L, 0L, 0L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) + } else { None } SparkListenerExecutorMetricsUpdate(execId, taskMetrics, executorMetricsUpdate) @@ -2082,7 +2084,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "OnHeapUnifiedMemory" : 432, | "OffHeapUnifiedMemory" : 321, | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765 + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 | } | |} @@ -2105,7 +2113,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "OnHeapUnifiedMemory" : 432, | "OffHeapUnifiedMemory" : 321, | "DirectPoolMemory" : 654, - | "MappedPoolMemory" : 765 + | "MappedPoolMemory" : 765, + | "ProcessTreeJVMVMemory": 256912, + | "ProcessTreeJVMRSSMemory": 123456, + | "ProcessTreePythonVMemory": 123456, + | "ProcessTreePythonRSSMemory": 61728, + | "ProcessTreeOtherVMemory": 30364, + | "ProcessTreeOtherRSSMemory": 15182 | } |} """.stripMargin diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 777950016801..8239cbc3a381 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -82,6 +82,8 @@ app-20161115172038-0000 app-20161116163331-0000 application_1516285256255_0012 application_1506645932520_24630151 +application_1538416563558_0014 +stat local-1422981759269 local-1422981780767 local-1425081759269