diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 97eed611e8f9a78f40d253172455f4146d06a0a4..9fe97b4d9c20ce901aca7059974121658f5c795f 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -113,6 +113,26 @@ public class SparkFirehoseListener implements SparkListenerInterface { onEvent(executorRemoved); } + @Override + public final void onExecutorBlacklisted(SparkListenerExecutorBlacklisted executorBlacklisted) { + onEvent(executorBlacklisted); + } + + @Override + public final void onExecutorUnblacklisted(SparkListenerExecutorUnblacklisted executorUnblacklisted) { + onEvent(executorUnblacklisted); + } + + @Override + public final void onNodeBlacklisted(SparkListenerNodeBlacklisted nodeBlacklisted) { + onEvent(nodeBlacklisted); + } + + @Override + public final void onNodeUnblacklisted(SparkListenerNodeUnblacklisted nodeUnblacklisted) { + onEvent(nodeUnblacklisted); + } + @Override public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { onEvent(blockUpdated); diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 64ea719141f4b8145d2d3b7b451da5658ad98647..4e83d6d564986fc916c1bb8c491e913a6102876b 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -45,6 +45,11 @@ limitations under the License. title="Bytes and records written to disk in order to be read by a shuffle in a future stage."> Shuffle Write</span> </th> + <th> + <span data-toggle="tooltip" data-placement="left" + title="Number of executors blacklisted by the scheduler due to task failures."> + Blacklisted</span> + </th> </thead> <tbody> </tbody> diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index fe5db6aa26b659f51737ca1b321baa435d3f6672..7dbfe32de903a03bae6c25c713fb8a0250c73470 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -182,7 +182,7 @@ $(document).ready(function () { executorsSummary = $("#active-executors"); getStandAloneppId(function (appId) { - + var endPoint = createRESTEndPoint(appId); $.getJSON(endPoint, function (response, status, jqXHR) { var summary = []; @@ -202,7 +202,8 @@ $(document).ready(function () { var allTotalInputBytes = 0; var allTotalShuffleRead = 0; var allTotalShuffleWrite = 0; - + var allTotalBlacklisted = 0; + var activeExecCnt = 0; var activeRDDBlocks = 0; var activeMemoryUsed = 0; @@ -219,7 +220,8 @@ $(document).ready(function () { var activeTotalInputBytes = 0; var activeTotalShuffleRead = 0; var activeTotalShuffleWrite = 0; - + var activeTotalBlacklisted = 0; + var deadExecCnt = 0; var deadRDDBlocks = 0; var deadMemoryUsed = 0; @@ -236,7 +238,8 @@ $(document).ready(function () { var deadTotalInputBytes = 0; var deadTotalShuffleRead = 0; var deadTotalShuffleWrite = 0; - + var deadTotalBlacklisted = 0; + response.forEach(function (exec) { allExecCnt += 1; allRDDBlocks += exec.rddBlocks; @@ -254,6 +257,7 @@ $(document).ready(function () { allTotalInputBytes += exec.totalInputBytes; allTotalShuffleRead += exec.totalShuffleRead; allTotalShuffleWrite += exec.totalShuffleWrite; + allTotalBlacklisted += exec.isBlacklisted ? 1 : 0; if (exec.isActive) { activeExecCnt += 1; activeRDDBlocks += exec.rddBlocks; @@ -271,6 +275,7 @@ $(document).ready(function () { activeTotalInputBytes += exec.totalInputBytes; activeTotalShuffleRead += exec.totalShuffleRead; activeTotalShuffleWrite += exec.totalShuffleWrite; + activeTotalBlacklisted += exec.isBlacklisted ? 1 : 0; } else { deadExecCnt += 1; deadRDDBlocks += exec.rddBlocks; @@ -288,9 +293,10 @@ $(document).ready(function () { deadTotalInputBytes += exec.totalInputBytes; deadTotalShuffleRead += exec.totalShuffleRead; deadTotalShuffleWrite += exec.totalShuffleWrite; + deadTotalBlacklisted += exec.isBlacklisted ? 1 : 0; } }); - + var totalSummary = { "execCnt": ( "Total(" + allExecCnt + ")"), "allRDDBlocks": allRDDBlocks, @@ -307,7 +313,8 @@ $(document).ready(function () { "allTotalGCTime": allTotalGCTime, "allTotalInputBytes": allTotalInputBytes, "allTotalShuffleRead": allTotalShuffleRead, - "allTotalShuffleWrite": allTotalShuffleWrite + "allTotalShuffleWrite": allTotalShuffleWrite, + "allTotalBlacklisted": allTotalBlacklisted }; var activeSummary = { "execCnt": ( "Active(" + activeExecCnt + ")"), @@ -325,7 +332,8 @@ $(document).ready(function () { "allTotalGCTime": activeTotalGCTime, "allTotalInputBytes": activeTotalInputBytes, "allTotalShuffleRead": activeTotalShuffleRead, - "allTotalShuffleWrite": activeTotalShuffleWrite + "allTotalShuffleWrite": activeTotalShuffleWrite, + "allTotalBlacklisted": activeTotalBlacklisted }; var deadSummary = { "execCnt": ( "Dead(" + deadExecCnt + ")" ), @@ -343,12 +351,13 @@ $(document).ready(function () { "allTotalGCTime": deadTotalGCTime, "allTotalInputBytes": deadTotalInputBytes, "allTotalShuffleRead": deadTotalShuffleRead, - "allTotalShuffleWrite": deadTotalShuffleWrite + "allTotalShuffleWrite": deadTotalShuffleWrite, + "allTotalBlacklisted": deadTotalBlacklisted }; - + var data = {executors: response, "execSummary": [activeSummary, deadSummary, totalSummary]}; $.get(createTemplateURI(appId), function (template) { - + executorsSummary.append(Mustache.render($(template).filter("#executors-summary-template").html(), data)); var selector = "#active-executors-table"; var conf = { @@ -360,7 +369,12 @@ $(document).ready(function () { } }, {data: 'hostPort'}, - {data: 'isActive', render: formatStatus}, + {data: 'isActive', render: function (data, type, row) { + if (type !== 'display') return data; + if (row.isBlacklisted) return "Blacklisted"; + else return formatStatus (data, type); + } + }, {data: 'rddBlocks'}, { data: function (row, type) { @@ -474,7 +488,8 @@ $(document).ready(function () { }, {data: 'allTotalInputBytes', render: formatBytes}, {data: 'allTotalShuffleRead', render: formatBytes}, - {data: 'allTotalShuffleWrite', render: formatBytes} + {data: 'allTotalShuffleWrite', render: formatBytes}, + {data: 'allTotalBlacklisted'} ], "paging": false, "searching": false, diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index bf7a62ea33875f4c3f23733dec6748342406241c..77d5c97a32711af6268a2294440384c16c921477 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.util.{Clock, SystemClock, Utils} @@ -48,9 +48,14 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} * one exception is [[nodeBlacklist()]], which can be called without holding a lock. */ private[scheduler] class BlacklistTracker ( + private val listenerBus: LiveListenerBus, conf: SparkConf, clock: Clock = new SystemClock()) extends Logging { + def this(sc: SparkContext) = { + this(sc.listenerBus, sc.conf) + } + BlacklistTracker.validateBlacklistConfs(conf) private val MAX_FAILURES_PER_EXEC = conf.get(config.MAX_FAILURES_PER_EXEC) private val MAX_FAILED_EXEC_PER_NODE = conf.get(config.MAX_FAILED_EXEC_PER_NODE) @@ -103,6 +108,7 @@ private[scheduler] class BlacklistTracker ( execsToUnblacklist.foreach { exec => val status = executorIdToBlacklistStatus.remove(exec).get val failedExecsOnNode = nodeToBlacklistedExecs(status.node) + listenerBus.post(SparkListenerExecutorUnblacklisted(now, exec)) failedExecsOnNode.remove(exec) if (failedExecsOnNode.isEmpty) { nodeToBlacklistedExecs.remove(status.node) @@ -114,7 +120,10 @@ private[scheduler] class BlacklistTracker ( // Un-blacklist any nodes that have been blacklisted longer than the blacklist timeout. logInfo(s"Removing nodes $nodesToUnblacklist from blacklist because the blacklist " + s"has timed out") - nodeIdToBlacklistExpiryTime --= nodesToUnblacklist + nodesToUnblacklist.foreach { node => + nodeIdToBlacklistExpiryTime.remove(node) + listenerBus.post(SparkListenerNodeUnblacklisted(now, node)) + } _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } updateNextExpiryTime() @@ -161,6 +170,8 @@ private[scheduler] class BlacklistTracker ( s" task failures in successful task sets") val node = failuresInTaskSet.node executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTimeForNewBlacklists)) + listenerBus.post(SparkListenerExecutorBlacklisted(now, exec, newTotal)) + executorIdToFailureList.remove(exec) updateNextExpiryTime() // In addition to blacklisting the executor, we also update the data for failures on the @@ -174,6 +185,7 @@ private[scheduler] class BlacklistTracker ( logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " + s"executors blacklisted: ${blacklistedExecsOnNode}") nodeIdToBlacklistExpiryTime.put(node, expiryTimeForNewBlacklists) + listenerBus.post(SparkListenerNodeBlacklisted(now, node, blacklistedExecsOnNode.size)) _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index f39565edd2358187e670d7eaf4ceb138e0157d24..af9bdefc967efe2766f9aa9a8076f41e861be618 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -193,6 +193,22 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) } + override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onNodeBlacklisted(event: SparkListenerNodeBlacklisted): Unit = { + logEvent(event, flushLogger = true) + } + + override def onNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Unit = { + logEvent(event, flushLogger = true) + } + // No-op because logging every update would be overkill override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {} diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 7618dfeeedf8d40dba8942fdba31eeeea028771d..1b12af75864eaf9a9369d80ad46c25f1d1acf9b4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -105,6 +105,28 @@ case class SparkListenerExecutorAdded(time: Long, executorId: String, executorIn case class SparkListenerExecutorRemoved(time: Long, executorId: String, reason: String) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerExecutorBlacklisted( + time: Long, + executorId: String, + taskFailures: Int) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerExecutorUnblacklisted(time: Long, executorId: String) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerNodeBlacklisted( + time: Long, + hostId: String, + executorFailures: Int) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerNodeUnblacklisted(time: Long, hostId: String) + extends SparkListenerEvent + @DeveloperApi case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends SparkListenerEvent @@ -238,6 +260,26 @@ private[spark] trait SparkListenerInterface { */ def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit + /** + * Called when the driver blacklists an executor for a Spark application. + */ + def onExecutorBlacklisted(executorBlacklisted: SparkListenerExecutorBlacklisted): Unit + + /** + * Called when the driver re-enables a previously blacklisted executor. + */ + def onExecutorUnblacklisted(executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit + + /** + * Called when the driver blacklists a node for a Spark application. + */ + def onNodeBlacklisted(nodeBlacklisted: SparkListenerNodeBlacklisted): Unit + + /** + * Called when the driver re-enables a previously blacklisted node. + */ + def onNodeUnblacklisted(nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit + /** * Called when the driver receives a block update info. */ @@ -293,6 +335,18 @@ abstract class SparkListener extends SparkListenerInterface { override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { } + override def onExecutorBlacklisted( + executorBlacklisted: SparkListenerExecutorBlacklisted): Unit = { } + + override def onExecutorUnblacklisted( + executorUnblacklisted: SparkListenerExecutorUnblacklisted): Unit = { } + + override def onNodeBlacklisted( + nodeBlacklisted: SparkListenerNodeBlacklisted): Unit = { } + + override def onNodeUnblacklisted( + nodeUnblacklisted: SparkListenerNodeUnblacklisted): Unit = { } + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { } override def onOtherEvent(event: SparkListenerEvent): Unit = { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 471586ac0852a4a0d27b081422b660f422f7cbb2..3ff363321e8c9cf42ed8e60c66641560ea80e354 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,6 +61,14 @@ private[spark] trait SparkListenerBus listener.onExecutorAdded(executorAdded) case executorRemoved: SparkListenerExecutorRemoved => listener.onExecutorRemoved(executorRemoved) + case executorBlacklisted: SparkListenerExecutorBlacklisted => + listener.onExecutorBlacklisted(executorBlacklisted) + case executorUnblacklisted: SparkListenerExecutorUnblacklisted => + listener.onExecutorUnblacklisted(executorUnblacklisted) + case nodeBlacklisted: SparkListenerNodeBlacklisted => + listener.onNodeBlacklisted(nodeBlacklisted) + case nodeUnblacklisted: SparkListenerNodeUnblacklisted => + listener.onNodeUnblacklisted(nodeUnblacklisted) case blockUpdated: SparkListenerBlockUpdated => listener.onBlockUpdated(blockUpdated) case logStart: SparkListenerLogStart => // ignore event log metadata diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 9a8e313f9ec7b854b828e286d3786b9a21f70278..72ed55af41f4de2e3c8cdd0f81cfd182609c1cd8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -63,14 +63,14 @@ private[spark] class TaskSchedulerImpl private[scheduler]( this( sc, sc.conf.get(config.MAX_TASK_FAILURES), - TaskSchedulerImpl.maybeCreateBlacklistTracker(sc.conf)) + TaskSchedulerImpl.maybeCreateBlacklistTracker(sc)) } def this(sc: SparkContext, maxTaskFailures: Int, isLocal: Boolean) = { this( sc, maxTaskFailures, - TaskSchedulerImpl.maybeCreateBlacklistTracker(sc.conf), + TaskSchedulerImpl.maybeCreateBlacklistTracker(sc), isLocal = isLocal) } @@ -717,9 +717,9 @@ private[spark] object TaskSchedulerImpl { retval.toList } - private def maybeCreateBlacklistTracker(conf: SparkConf): Option[BlacklistTracker] = { - if (BlacklistTracker.isBlacklistEnabled(conf)) { - Some(new BlacklistTracker(conf)) + private def maybeCreateBlacklistTracker(sc: SparkContext): Option[BlacklistTracker] = { + if (BlacklistTracker.isBlacklistEnabled(sc.conf)) { + Some(new BlacklistTracker(sc)) } else { None } 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 44a929b31038463ccadbf124d3e8601912900e85..7d035b11fa422c6d24824f3dcf267c9240ef9423 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 @@ -73,6 +73,7 @@ class ExecutorSummary private[spark]( val totalInputBytes: Long, val totalShuffleRead: Long, val totalShuffleWrite: Long, + val isBlacklisted: Boolean, val maxMemory: Long, val executorLogs: Map[String, String]) diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 3cc5353f475f490990567fd87bb0f4382d2e6569..766cc65084f073bb677f38d8be52f203bec26e56 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -91,6 +91,9 @@ private[spark] object ToolTips { val TASK_TIME = "Shaded red when garbage collection (GC) time is over 10% of task time" + val BLACKLISTED = + "Shows if this executor has been blacklisted by the scheduler due to task failures." + val APPLICATION_EXECUTOR_LIMIT = """Maximum number of executors that this application will use. This limit is finite only when dynamic allocation is enabled. The number of granted executors may exceed the limit diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 7953d77fd7ece33434fac0be1d7fca7a1cb59343..2d1691e55c428466daea7900f391e9611a512d22 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -39,6 +39,7 @@ private[ui] case class ExecutorSummaryInfo( totalInputBytes: Long, totalShuffleRead: Long, totalShuffleWrite: Long, + isBlacklisted: Int, maxMemory: Long, executorLogs: Map[String, String]) @@ -101,6 +102,7 @@ private[spark] object ExecutorsPage { taskSummary.inputBytes, taskSummary.shuffleRead, taskSummary.shuffleWrite, + taskSummary.isBlacklisted, maxMem, taskSummary.executorLogs ) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 678571fd4f5acea89776b85336c09ad69c7a8176..8ae712f8ed323642eb157ff690636f6526d866b2 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -53,7 +53,8 @@ private[ui] case class ExecutorTaskSummary( var shuffleRead: Long = 0L, var shuffleWrite: Long = 0L, var executorLogs: Map[String, String] = Map.empty, - var isAlive: Boolean = true + var isAlive: Boolean = true, + var isBlacklisted: Boolean = false ) /** @@ -73,7 +74,8 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar def deadStorageStatusList: Seq[StorageStatus] = storageStatusListener.deadStorageStatusList - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized { + override def onExecutorAdded( + executorAdded: SparkListenerExecutorAdded): Unit = synchronized { val eid = executorAdded.executorId val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) taskSummary.executorLogs = executorAdded.executorInfo.logUrlMap @@ -100,7 +102,8 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar executorToTaskSummary.get(executorRemoved.executorId).foreach(e => e.isAlive = false) } - override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { + override def onApplicationStart( + applicationStart: SparkListenerApplicationStart): Unit = { applicationStart.driverLogs.foreach { logs => val storageStatus = activeStorageStatusList.find { s => s.blockManagerId.executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER || @@ -114,13 +117,15 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar } } - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { + override def onTaskStart( + taskStart: SparkListenerTaskStart): Unit = synchronized { val eid = taskStart.taskInfo.executorId val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) taskSummary.tasksActive += 1 } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + override def onTaskEnd( + taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo if (info != null) { val eid = info.executorId @@ -157,4 +162,46 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar } } + private def updateExecutorBlacklist( + eid: String, + isBlacklisted: Boolean): Unit = { + val execTaskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + execTaskSummary.isBlacklisted = isBlacklisted + } + + override def onExecutorBlacklisted( + executorBlacklisted: SparkListenerExecutorBlacklisted) + : Unit = synchronized { + updateExecutorBlacklist(executorBlacklisted.executorId, true) + } + + override def onExecutorUnblacklisted( + executorUnblacklisted: SparkListenerExecutorUnblacklisted) + : Unit = synchronized { + updateExecutorBlacklist(executorUnblacklisted.executorId, false) + } + + override def onNodeBlacklisted( + nodeBlacklisted: SparkListenerNodeBlacklisted) + : Unit = synchronized { + // Implicitly blacklist every executor associated with this node, and show this in the UI. + activeStorageStatusList.foreach { status => + if (status.blockManagerId.host == nodeBlacklisted.hostId) { + updateExecutorBlacklist(status.blockManagerId.executorId, true) + } + } + } + + override def onNodeUnblacklisted( + nodeUnblacklisted: SparkListenerNodeUnblacklisted) + : Unit = synchronized { + // Implicitly unblacklist every executor associated with this node, regardless of how + // they may have been blacklisted initially (either explicitly through executor blacklisting + // or implicitly through node blacklisting). Show this in the UI. + activeStorageStatusList.foreach { status => + if (status.blockManagerId.host == nodeUnblacklisted.hostId) { + updateExecutorBlacklist(status.blockManagerId.executorId, false) + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 9fb3f35fd96852e5a504fb372630d571114868e3..cd1b02addc789c60d13cb7ade6efc6024a8378f1 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -85,6 +85,11 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage <th>Shuffle Spill (Memory)</th> <th>Shuffle Spill (Disk)</th> }} + <th> + <span data-toggle="tooltip" title={ToolTips.BLACKLISTED}> + Blacklisted + </span> + </th> </thead> <tbody> {createExecutorTable()} @@ -160,6 +165,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage {Utils.bytesToString(v.diskBytesSpilled)} </td> }} + <td>{v.isBlacklisted}</td> </tr> } case None => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 9ce8542f0279131b3e7eb3b5059d55bd473b7bd6..371dad966d3cc9ea1d381077f032cd01cb2a40c2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -43,6 +43,7 @@ private[spark] object UIData { var shuffleWriteRecords : Long = 0 var memoryBytesSpilled : Long = 0 var diskBytesSpilled : Long = 0 + var isBlacklisted : Int = 0 } class JobUIData( @@ -92,6 +93,7 @@ private[spark] object UIData { var shuffleWriteRecords: Long = _ var memoryBytesSpilled: Long = _ var diskBytesSpilled: Long = _ + var isBlacklisted: Int = _ var schedulingPool: String = "" var description: Option[String] = None 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 cba44c848e0128c30ec1ddbf9b3711de476fbea5..10902ab5a83270584abd92fd3293fb7a74cb9306 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,32 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { @@ -8,8 +36,8 @@ "duration" : 10505, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 } ] }, { @@ -23,8 +51,8 @@ "duration" : 57, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -34,8 +62,8 @@ "duration" : 10, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, "lastUpdatedEpoch" : 0 } ] }, { @@ -49,8 +77,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426633910242, "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -60,8 +88,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, "lastUpdatedEpoch" : 0 } ] }, { @@ -74,8 +102,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1425081758277, "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, "lastUpdatedEpoch" : 0 } ] }, { @@ -88,8 +116,8 @@ "duration" : 9011, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981779720, "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, "lastUpdatedEpoch" : 0 } ] }, { @@ -102,8 +130,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index cba44c848e0128c30ec1ddbf9b3711de476fbea5..10902ab5a83270584abd92fd3293fb7a74cb9306 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,32 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { @@ -8,8 +36,8 @@ "duration" : 10505, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 } ] }, { @@ -23,8 +51,8 @@ "duration" : 57, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -34,8 +62,8 @@ "duration" : 10, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, "lastUpdatedEpoch" : 0 } ] }, { @@ -49,8 +77,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426633910242, "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -60,8 +88,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, "lastUpdatedEpoch" : 0 } ] }, { @@ -74,8 +102,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1425081758277, "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, "lastUpdatedEpoch" : 0 } ] }, { @@ -88,8 +116,8 @@ "duration" : 9011, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981779720, "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, "lastUpdatedEpoch" : 0 } ] }, { @@ -102,8 +130,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index e7db6742c25e1e4a5515b1652909a1f507328613..6b9f29e1a230e49ba5bbe2bfdb08b8132567d1be 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -16,6 +16,7 @@ "totalInputBytes" : 28000288, "totalShuffleRead" : 0, "totalShuffleWrite" : 13180, + "isBlacklisted" : false, "maxMemory" : 278302556, "executorLogs" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json new file mode 100644 index 0000000000000000000000000000000000000000..5914a1c2c4b6d9c8d395b61282fa7ba57f8df543 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -0,0 +1,118 @@ +[ { + "id" : "2", + "hostPort" : "172.22.0.167:51487", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2537, + "totalGCTime" : 88, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr" + } +}, { + "id" : "driver", + "hostPort" : "172.22.0.167:51475", + "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" : true, + "maxMemory" : 384093388, + "executorLogs" : { } +}, { + "id" : "1", + "hostPort" : "172.22.0.167:51490", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3152, + "totalGCTime" : 68, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout", + "stderr" : "http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr" + } +}, { + "id" : "0", + "hostPort" : "172.22.0.167:51491", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2551, + "totalGCTime" : 116, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr" + } +}, { + "id" : "3", + "hostPort" : "172.22.0.167:51485", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2453, + "totalGCTime" : 72, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : true, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr" + } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json new file mode 100644 index 0000000000000000000000000000000000000000..92e249c8511167b765f71f1ddbd24f098bf8d0de --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -0,0 +1,118 @@ +[ { + "id" : "2", + "hostPort" : "172.22.0.111:64539", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 6, + "completedTasks" : 0, + "totalTasks" : 6, + "totalDuration" : 2792, + "totalGCTime" : 128, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", + "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" + } +}, { + "id" : "driver", + "hostPort" : "172.22.0.111:64527", + "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, + "executorLogs" : { } +}, { + "id" : "1", + "hostPort" : "172.22.0.111:64541", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 12, + "totalTasks" : 12, + "totalDuration" : 2613, + "totalGCTime" : 84, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout", + "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" + } +}, { + "id" : "0", + "hostPort" : "172.22.0.111:64540", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 4, + "completedTasks" : 0, + "totalTasks" : 4, + "totalDuration" : 2741, + "totalGCTime" : 120, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", + "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" + } +}, { + "id" : "3", + "hostPort" : "172.22.0.111:64543", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 4, + "totalTasks" : 4, + "totalDuration" : 3457, + "totalGCTime" : 72, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "executorLogs" : { + "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", + "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" + } +} ] 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 9165f549d7d25a102ccf1fa9213e0cc02d4d103f..8820c717f85d99633e38ae62c207f5e231f8ae09 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,67 +1,43 @@ [ { - "id" : "local-1430917381534", + "id" : "app-20161116163331-0000", "name" : "Spark shell", "attempts" : [ { - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-05-06T13:03:11.398GMT", + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", "lastUpdated" : "", - "duration" : 10505, - "sparkUser" : "irashid", + "duration" : 10671, + "sparkUser" : "jose", "completed" : true, - "startTimeEpoch" : 1430917380893, - "endTimeEpoch" : 1430917391398, + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, "lastUpdatedEpoch" : 0 } ] }, { - "id" : "local-1430917381535", + "id" : "app-20161115172038-0000", "name" : "Spark shell", "attempts" : [ { - "attemptId" : "2", - "startTime" : "2015-05-06T13:03:00.893GMT", - "endTime" : "2015-05-06T13:03:00.950GMT", - "lastUpdated" : "", - "duration" : 57, - "sparkUser" : "irashid", - "completed" : true, - "startTimeEpoch" : 1430917380893, - "endTimeEpoch" : 1430917380950, - "lastUpdatedEpoch" : 0 - }, { - "attemptId" : "1", - "startTime" : "2015-05-06T13:03:00.880GMT", - "endTime" : "2015-05-06T13:03:00.890GMT", + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", "lastUpdated" : "", - "duration" : 10, - "sparkUser" : "irashid", + "duration" : 101795, + "sparkUser" : "jose", "completed" : true, - "startTimeEpoch" : 1430917380880, - "endTimeEpoch" : 1430917380890, + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, "lastUpdatedEpoch" : 0 } ] }, { - "id" : "local-1426533911241", + "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { - "attemptId" : "2", - "startTime" : "2015-03-17T23:11:50.242GMT", - "endTime" : "2015-03-17T23:12:25.177GMT", - "lastUpdated" : "", - "duration" : 34935, - "sparkUser" : "irashid", - "completed" : true, - "startTimeEpoch" : 1426633910242, - "endTimeEpoch" : 1426633945177, - "lastUpdatedEpoch" : 0 - }, { - "attemptId" : "1", - "startTime" : "2015-03-16T19:25:10.242GMT", - "endTime" : "2015-03-16T19:25:45.177GMT", + "startTime" : "2015-05-06T13:03:00.893GMT", + "endTime" : "2015-05-06T13:03:11.398GMT", "lastUpdated" : "", - "duration" : 34935, + "duration" : 10505, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, - "endTimeEpoch" : 1426533945177, + "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json index a525d61543a88691a45cacf268872adafe4fdeb0..c3fe4db222ae6875f453a50f17fe8d5900fca548 100644 --- a/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json @@ -8,8 +8,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json index cc567f66f02e80865cb2fe558befe8bb2ded70e0..8281fa75aa0db0fe842baf6624cd6c397f7a1f90 100644 --- a/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json @@ -8,8 +8,8 @@ "duration" : 9011, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981779720, "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, "lastUpdatedEpoch" : 0 } ] }, { @@ -22,8 +22,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912, + "startTimeEpoch" : 1422981758277, "lastUpdatedEpoch" : 0 } ] } ] 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 c934a871724b59b28eb05239481bf2035d745b98..1930281f1a3ec46b68ee7345112b1c7e3b0c58d9 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,32 @@ [ { + "id" : "app-20161116163331-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-16T22:33:29.916GMT", + "endTime" : "2016-11-16T22:33:40.587GMT", + "lastUpdated" : "", + "duration" : 10671, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479335620587, + "startTimeEpoch" : 1479335609916, + "lastUpdatedEpoch" : 0 + } ] +}, { + "id" : "app-20161115172038-0000", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2016-11-15T23:20:37.079GMT", + "endTime" : "2016-11-15T23:22:18.874GMT", + "lastUpdated" : "", + "duration" : 101795, + "sparkUser" : "jose", + "completed" : true, + "endTimeEpoch" : 1479252138874, + "startTimeEpoch" : 1479252037079, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "local-1430917381534", "name" : "Spark shell", "attempts" : [ { @@ -8,8 +36,8 @@ "duration" : 10505, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917391398, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 } ] }, { @@ -23,8 +51,8 @@ "duration" : 57, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380893, "endTimeEpoch" : 1430917380950, + "startTimeEpoch" : 1430917380893, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -34,8 +62,8 @@ "duration" : 10, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890, + "startTimeEpoch" : 1430917380880, "lastUpdatedEpoch" : 0 } ] }, { @@ -49,8 +77,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426633910242, "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -60,8 +88,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, "lastUpdatedEpoch" : 0 } ] }, { @@ -74,8 +102,8 @@ "duration" : 8635, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1425081758277, "endTimeEpoch" : 1425081766912, + "startTimeEpoch" : 1425081758277, "lastUpdatedEpoch" : 0 } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json index f486d46313d8b9d12c4fac2e0efdf2ded453ddb9..e8ed96dc85f095759ef05431e46c280e0c157f87 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json @@ -8,8 +8,8 @@ "duration" : 9011, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1422981779720, "endTimeEpoch" : 1422981788731, + "startTimeEpoch" : 1422981779720, "lastUpdatedEpoch" : 0 } ] } diff --git a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json index e63039f6a17fc4cfb18ef10082e440e91dd4d672..88c601512d79ff44bbc4eee9aacff6d215ea1db1 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json @@ -9,8 +9,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426633910242, "endTimeEpoch" : 1426633945177, + "startTimeEpoch" : 1426633910242, "lastUpdatedEpoch" : 0 }, { "attemptId" : "1", @@ -20,8 +20,8 @@ "duration" : 34935, "sparkUser" : "irashid", "completed" : true, - "startTimeEpoch" : 1426533910242, "endTimeEpoch" : 1426533945177, + "startTimeEpoch" : 1426533910242, "lastUpdatedEpoch" : 0 } ] } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 0084339d2464252776afd90813d03bda59929352..cdebb5f5b91bc28a59f0460eb49dbd437027952c 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -68,9 +68,9 @@ } } }, - "11" : { - "taskId" : 11, - "index" : 3, + "9" : { + "taskId" : 9, + "index" : 1, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "executorId" : "<driver>", @@ -79,13 +79,13 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 1, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 434, + "executorRunTime" : 436, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -105,17 +105,17 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, + "bytesWritten" : 1648, + "writeTime" : 98000, "recordsWritten" : 0 } } }, - "14" : { - "taskId" : 14, - "index" : 6, + "10" : { + "taskId" : 10, + "index" : 2, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -149,16 +149,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 88000, + "writeTime" : 76000, "recordsWritten" : 0 } } }, - "13" : { - "taskId" : 13, - "index" : 5, + "11" : { + "taskId" : 11, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -171,7 +171,7 @@ "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -191,17 +191,17 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, + "bytesWritten" : 1647, + "writeTime" : 83000, "recordsWritten" : 0 } } }, - "10" : { - "taskId" : 10, - "index" : 2, + "12" : { + "taskId" : 12, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -234,30 +234,30 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 76000, + "bytesWritten" : 1645, + "writeTime" : 101000, "recordsWritten" : 0 } } }, - "9" : { - "taskId" : 9, - "index" : 1, + "13" : { + "taskId" : 13, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, + "executorDeserializeTime" : 2, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 436, + "executorRunTime" : 434, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -278,16 +278,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 98000, + "writeTime" : 73000, "recordsWritten" : 0 } } }, - "12" : { - "taskId" : 12, - "index" : 4, + "14" : { + "taskId" : 14, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.832GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -320,8 +320,8 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1645, - "writeTime" : 101000, + "bytesWritten" : 1648, + "writeTime" : 88000, "recordsWritten" : 0 } } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 63fe3b2f958e573282703103516516d9007185ed..85dafcffc87eab0af79e209cbd07de102cfe5692 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -68,9 +68,9 @@ } } }, - "11" : { - "taskId" : 11, - "index" : 3, + "9" : { + "taskId" : 9, + "index" : 1, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "executorId" : "<driver>", @@ -79,13 +79,13 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 1, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 434, + "executorRunTime" : 436, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -105,17 +105,17 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, + "bytesWritten" : 1648, + "writeTime" : 98000, "recordsWritten" : 0 } } }, - "14" : { - "taskId" : 14, - "index" : 6, + "10" : { + "taskId" : 10, + "index" : 2, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -149,16 +149,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 88000, + "writeTime" : 76000, "recordsWritten" : 0 } } }, - "13" : { - "taskId" : 13, - "index" : 5, + "11" : { + "taskId" : 11, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.830GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -171,7 +171,7 @@ "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -191,17 +191,17 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, + "bytesWritten" : 1647, + "writeTime" : 83000, "recordsWritten" : 0 } } }, - "10" : { - "taskId" : 10, - "index" : 2, + "12" : { + "taskId" : 12, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -234,30 +234,30 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 76000, + "bytesWritten" : 1645, + "writeTime" : 101000, "recordsWritten" : 0 } } }, - "9" : { - "taskId" : 9, - "index" : 1, + "13" : { + "taskId" : 13, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", + "launchTime" : "2015-02-03T16:43:05.831GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, + "executorDeserializeTime" : 2, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 436, + "executorRunTime" : 434, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 0, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -278,16 +278,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 98000, + "writeTime" : 73000, "recordsWritten" : 0 } } }, - "12" : { - "taskId" : 12, - "index" : 4, + "14" : { + "taskId" : 14, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", + "launchTime" : "2015-02-03T16:43:05.832GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -320,8 +320,8 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1645, - "writeTime" : 101000, + "bytesWritten" : 1648, + "writeTime" : 88000, "recordsWritten" : 0 } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json index 5dcbc890438b2e815161f970fc99a8a4bd8bbff1..0ed609d5b7f9247702da4bbc8820bdef3acfe497 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json @@ -3,7 +3,7 @@ "executorDeserializeTime" : [ 1.0, 3.0, 36.0 ], "executorDeserializeCpuTime" : [ 0.0, 0.0, 0.0 ], "executorRunTime" : [ 16.0, 28.0, 351.0 ], - "executorCpuTime" : [ 0.0, 0.0, 0.0], + "executorCpuTime" : [ 0.0, 0.0, 0.0 ], "resultSize" : [ 2010.0, 2065.0, 2065.0 ], "jvmGcTime" : [ 0.0, 0.0, 7.0 ], "resultSerializationTime" : [ 0.0, 0.0, 2.0 ], diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index aaeef1f2f582c908f6f3539c47debc9db0d596cd..c2ece6f4d0d74d15fbc97285a4506337c83528c6 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -29,11 +29,11 @@ "value" : "5050" } ], "tasks" : { - "2" : { - "taskId" : 2, - "index" : 2, + "0" : { + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", + "launchTime" : "2015-03-16T19:25:36.515GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -41,11 +41,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "378", - "value" : "378" + "update" : "78", + "value" : "5050" } ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 14, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -77,11 +77,11 @@ } } }, - "5" : { - "taskId" : 5, - "index" : 5, + "1" : { + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", + "launchTime" : "2015-03-16T19:25:36.521GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -89,11 +89,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "897", - "value" : "3750" + "update" : "247", + "value" : "2175" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 14, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -125,9 +125,9 @@ } } }, - "4" : { - "taskId" : 4, - "index" : 4, + "2" : { + "taskId" : 2, + "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "executorId" : "<driver>", @@ -137,17 +137,17 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "678", - "value" : "2853" + "update" : "378", + "value" : "378" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 13, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -173,11 +173,11 @@ } } }, - "7" : { - "taskId" : 7, - "index" : 7, + "3" : { + "taskId" : 3, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.524GMT", + "launchTime" : "2015-03-16T19:25:36.522GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -185,11 +185,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "1222", - "value" : "4972" + "update" : "572", + "value" : "950" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 13, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -221,11 +221,11 @@ } } }, - "1" : { - "taskId" : 1, - "index" : 1, + "4" : { + "taskId" : 4, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.521GMT", + "launchTime" : "2015-03-16T19:25:36.522GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -233,17 +233,17 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "247", - "value" : "2175" + "update" : "678", + "value" : "2853" } ], "taskMetrics" : { - "executorDeserializeTime" : 14, + "executorDeserializeTime" : 12, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "inputMetrics" : { @@ -269,11 +269,11 @@ } } }, - "3" : { - "taskId" : 3, - "index" : 3, + "5" : { + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", + "launchTime" : "2015-03-16T19:25:36.523GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -281,11 +281,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "572", - "value" : "950" + "update" : "897", + "value" : "3750" } ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 12, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -365,11 +365,11 @@ } } }, - "0" : { - "taskId" : 0, - "index" : 0, + "7" : { + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.515GMT", + "launchTime" : "2015-03-16T19:25:36.524GMT", "executorId" : "<driver>", "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", @@ -377,11 +377,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "78", - "value" : "5050" + "update" : "1222", + "value" : "4972" } ], "taskMetrics" : { - "executorDeserializeTime" : 14, + "executorDeserializeTime" : 12, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, diff --git a/core/src/test/resources/spark-events/app-20161115172038-0000 b/core/src/test/resources/spark-events/app-20161115172038-0000 new file mode 100755 index 0000000000000000000000000000000000000000..3af0451d0c3928c488dcba941356ea3cea9d12e8 --- /dev/null +++ b/core/src/test/resources/spark-events/app-20161115172038-0000 @@ -0,0 +1,75 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.111","Port":64527},"Maximum Memory":384093388,"Timestamp":1479252038836} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.111","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"64511","spark.repl.class.uri":"spark://172.22.0.111:64511/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-f09ef9e2-7f15-433f-a5d1-30138d8764ca/repl-28d60911-dbc3-465f-b7b3-ee55c071595e","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"10000","spark.app.id":"app-20161115172038-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","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":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=10000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161115172038-0000","Timestamp":1479252037079,"User":"jose"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042589,"Executor ID":"2","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout","stderr":"http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042593,"Executor ID":"0","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout","stderr":"http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042629,"Executor ID":"1","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout","stderr":"http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"172.22.0.111","Port":64540},"Maximum Memory":384093388,"Timestamp":1479252042687} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"172.22.0.111","Port":64539},"Maximum Memory":384093388,"Timestamp":1479252042689} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"172.22.0.111","Port":64541},"Maximum Memory":384093388,"Timestamp":1479252042692} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479252042711,"Executor ID":"3","Executor Info":{"Host":"172.22.0.111","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.111","Port":64543},"Maximum Memory":384093388,"Timestamp":1479252042759} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1479252043855,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at <console>:26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:35)\n$line16.$read$$iw$$iw$$iw$$iw.<init>(<console>:37)\n$line16.$read$$iw$$iw$$iw.<init>(<console>:39)\n$line16.$read$$iw$$iw.<init>(<console>:41)\n$line16.$read$$iw.<init>(<console>:43)\n$line16.$read.<init>(<console>:45)\n$line16.$read$.<init>(<console>:49)\n$line16.$read$.<clinit>(<console>)\n$line16.$eval$.$print$lzycompute(<console>:7)\n$line16.$eval$.$print(<console>:6)\n$line16.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]}],"Stage IDs":[0],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at <console>:26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:35)\n$line16.$read$$iw$$iw$$iw$$iw.<init>(<console>:37)\n$line16.$read$$iw$$iw$$iw.<init>(<console>:39)\n$line16.$read$$iw$$iw.<init>(<console>:41)\n$line16.$read$$iw.<init>(<console>:43)\n$line16.$read.<init>(<console>:45)\n$line16.$read$.<init>(<console>:49)\n$line16.$read$.<clinit>(<console>)\n$line16.$eval$.$print$lzycompute(<console>:7)\n$line16.$eval$.$print(<console>:6)\n$line16.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479252044021,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479252044053,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479252044054,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479252044057,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479252044059,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479252044060,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479252044064,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479252044065,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479252044059,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044653,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":499,"Value":499,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":52390000,"Value":52390000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":7909000,"Value":7909000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":1123,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":21,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":499,"Executor Deserialize CPU Time":52390000,"Executor Run Time":18,"Executor CPU Time":7909000,"Result Size":1123,"JVM GC Time":21,"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,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479252044054,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044657,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":508,"Value":1007,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":36827000,"Value":89217000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":4333000,"Value":12242000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":2246,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":42,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":508,"Executor Deserialize CPU Time":36827000,"Executor Run Time":18,"Executor CPU Time":4333000,"Result Size":1123,"JVM GC Time":21,"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,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044658,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":509,"Value":1516,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":44100000,"Value":133317000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":17,"Value":53,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":11340000,"Value":23582000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":3369,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":509,"Executor Deserialize CPU Time":44100000,"Executor Run Time":17,"Executor CPU Time":11340000,"Result Size":1123,"JVM GC Time":21,"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,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479252044021,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044692,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":511,"Value":2027,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":227762000,"Value":361079000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":3631000,"Value":27213000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1938,"Value":5307,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":21,"Value":84,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":511,"Executor Deserialize CPU Time":227762000,"Executor Run Time":16,"Executor CPU Time":3631000,"Result Size":1938,"JVM GC Time":21,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":495,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044720,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":495,"Value":564,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":114,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":495,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044727,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Value":1058,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":144,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":494,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479252044060,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044729,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":494,"Value":1552,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":174,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":494,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":13,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":5,"Attempt":1,"Launch Time":1479252044732,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044736,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":2003,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":206,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":11,"Attempt":1,"Launch Time":1479252044736,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479252044065,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044737,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Value":2449,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":238,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":446,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":15,"Attempt":1,"Launch Time":1479252044737,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":448,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479252044056,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044741,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":448,"Value":2897,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":270,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":448,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":7,"Attempt":1,"Launch Time":1479252044742,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044752,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044748,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":2035,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3655000,"Value":364734000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":899000,"Value":28112000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":884,"Value":6191,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":3655000,"Executor Run Time":0,"Executor CPU Time":899000,"Result Size":884,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":19,"Index":11,"Attempt":1,"Launch Time":1479252044736,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044749,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":2899,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":2,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":11,"Attempt":2,"Launch Time":1479252044749,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":7,"Attempt":1,"Launch Time":1479252044742,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044752,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":2038,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3566000,"Value":368300000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2900,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1004000,"Value":29116000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7154,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3566000,"Executor Run Time":1,"Executor CPU Time":1004000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":10,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":20,"Index":15,"Attempt":1,"Launch Time":1479252044737,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044756,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":10,"Value":2910,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":10,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":15,"Attempt":2,"Launch Time":1479252044756,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":11,"Attempt":2,"Launch Time":1479252044749,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044759,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":2042,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3720000,"Value":372020000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2911,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1009000,"Value":30125000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8117,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3720000,"Executor Run Time":1,"Executor CPU Time":1009000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":5,"Attempt":1,"Launch Time":1479252044732,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044760,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":2047,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4303000,"Value":376323000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":2913,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":999000,"Value":31124000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9080,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4303000,"Executor Run Time":2,"Executor CPU Time":999000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":15,"Attempt":2,"Launch Time":1479252044756,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044768,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":2053,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4946000,"Value":381269000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":2914,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1176000,"Value":32300000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10043,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4946000,"Executor Run Time":1,"Executor CPU Time":1176000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":13,"Attempt":1,"Launch Time":1479252044731,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044775,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":2060,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3406000,"Value":384675000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1007000,"Value":33307000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":971,"Value":11014,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":6,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":3406000,"Executor Run Time":0,"Executor CPU Time":1007000,"Result Size":971,"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,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":456,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479252044053,"Executor ID":"2","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044778,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":456,"Value":3370,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":32,"Value":302,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":456,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":32,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":3,"Attempt":1,"Launch Time":1479252044778,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":503,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479252044057,"Executor ID":"0","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044789,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":503,"Value":3873,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":30,"Value":332,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":503,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":30,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":9,"Attempt":1,"Launch Time":1479252044789,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":3,"Attempt":1,"Launch Time":1479252044778,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044791,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":2065,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2950000,"Value":387625000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3875,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":822000,"Value":34129000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11977,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":2950000,"Executor Run Time":2,"Executor CPU Time":822000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":9,"Attempt":1,"Launch Time":1479252044789,"Executor ID":"1","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044798,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":2068,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2604000,"Value":390229000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3876,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":845000,"Value":34974000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12940,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2604000,"Executor Run Time":1,"Executor CPU Time":845000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479252044055,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044920,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":784,"Value":2852,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":56180000,"Value":446409000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":24,"Value":3900,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6046000,"Value":41020000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":13976,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":350,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":784,"Executor Deserialize CPU Time":56180000,"Executor Run Time":24,"Executor CPU Time":6046000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479252044052,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044921,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":789,"Value":3641,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":34766000,"Value":481175000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":3922,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":8189000,"Value":49209000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":15012,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":368,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":789,"Executor Deserialize CPU Time":34766000,"Executor Run Time":22,"Executor CPU Time":8189000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479252044064,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044921,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":777,"Value":4418,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29960000,"Value":511135000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":24,"Value":3946,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9708000,"Value":58917000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":16048,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":386,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":777,"Executor Deserialize CPU Time":29960000,"Executor Run Time":24,"Executor CPU Time":9708000,"Result Size":1036,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479252044058,"Executor ID":"3","Host":"172.22.0.111","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479252044924,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":791,"Value":5209,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":266560000,"Value":777695000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3962,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5884000,"Value":64801000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1851,"Value":17899,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":404,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":791,"Executor Deserialize CPU Time":266560000,"Executor Run Time":16,"Executor CPU Time":5884000,"Result Size":1851,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at <console>:26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:35)\n$line16.$read$$iw$$iw$$iw$$iw.<init>(<console>:37)\n$line16.$read$$iw$$iw$$iw.<init>(<console>:39)\n$line16.$read$$iw$$iw.<init>(<console>:41)\n$line16.$read$$iw.<init>(<console>:43)\n$line16.$read.<init>(<console>:45)\n$line16.$read$.<init>(<console>:49)\n$line16.$read$.<clinit>(<console>)\n$line16.$eval$.$print$lzycompute(<console>:7)\n$line16.$eval$.$print(<console>:6)\n$line16.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Submission Time":1479252044017,"Completion Time":1479252044926,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":3962,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":404,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":17899,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":777695000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":64801000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":6,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":5209,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1479252044931,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479252044930,"executorId":"2","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479252044930,"executorId":"0","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeBlacklisted","time":1479252044930,"hostId":"172.22.0.111","executorFailures":2} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted","time":1479252055635,"executorId":"2"} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted","time":1479252055635,"executorId":"0"} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeUnblacklisted","time":1479252055635,"hostId":"172.22.0.111"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1479252138874} diff --git a/core/src/test/resources/spark-events/app-20161116163331-0000 b/core/src/test/resources/spark-events/app-20161116163331-0000 new file mode 100755 index 0000000000000000000000000000000000000000..7566c9fc0a20b134e6572f90972056d064837f2a --- /dev/null +++ b/core/src/test/resources/spark-events/app-20161116163331-0000 @@ -0,0 +1,68 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.167","Port":51475},"Maximum Memory":384093388,"Timestamp":1479335611477} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.167","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"51459","spark.repl.class.uri":"spark://172.22.0.167:51459/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"1000000","spark.app.id":"app-20161116163331-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","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":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161116163331-0000","Timestamp":1479335609916,"User":"jose"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615320,"Executor ID":"3","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.167","Port":51485},"Maximum Memory":384093388,"Timestamp":1479335615387} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615393,"Executor ID":"2","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stdout","stderr":"http://172.22.0.167:51469/logPage/?appId=app-20161116163331-0000&executorId=2&logType=stderr"}}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615443,"Executor ID":"1","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stdout","stderr":"http://172.22.0.167:51467/logPage/?appId=app-20161116163331-0000&executorId=1&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"172.22.0.167","Port":51487},"Maximum Memory":384093388,"Timestamp":1479335615448} +{"Event":"SparkListenerExecutorAdded","Timestamp":1479335615462,"Executor ID":"0","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stdout","stderr":"http://172.22.0.167:51465/logPage/?appId=app-20161116163331-0000&executorId=0&logType=stderr"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"172.22.0.167","Port":51490},"Maximum Memory":384093388,"Timestamp":1479335615496} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"172.22.0.167","Port":51491},"Maximum Memory":384093388,"Timestamp":1479335615515} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1479335616467,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at <console>:26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:35)\n$line16.$read$$iw$$iw$$iw$$iw.<init>(<console>:37)\n$line16.$read$$iw$$iw$$iw.<init>(<console>:39)\n$line16.$read$$iw$$iw.<init>(<console>:41)\n$line16.$read$$iw.<init>(<console>:43)\n$line16.$read.<init>(<console>:45)\n$line16.$read$.<init>(<console>:49)\n$line16.$read$.<clinit>(<console>)\n$line16.$eval$.$print$lzycompute(<console>:7)\n$line16.$eval$.$print(<console>:6)\n$line16.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]}],"Stage IDs":[0],"Properties":{}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at <console>:26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:35)\n$line16.$read$$iw$$iw$$iw$$iw.<init>(<console>:37)\n$line16.$read$$iw$$iw$$iw.<init>(<console>:39)\n$line16.$read$$iw$$iw.<init>(<console>:41)\n$line16.$read$$iw.<init>(<console>:43)\n$line16.$read.<init>(<console>:45)\n$line16.$read$.<init>(<console>:49)\n$line16.$read$.<clinit>(<console>)\n$line16.$eval$.$print$lzycompute(<console>:7)\n$line16.$eval$.$print(<console>:6)\n$line16.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Accumulables":[]},"Properties":{}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479335616657,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479335616687,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479335616689,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479335616690,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479335616691,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479335616693,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479335616695,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479335616697,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617253,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":465,"Value":465,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":47305000,"Value":47305000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":7220000,"Value":7220000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":1123,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":465,"Executor Deserialize CPU Time":47305000,"Executor Run Time":22,"Executor CPU Time":7220000,"Result Size":1123,"JVM GC Time":18,"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,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1479335616697,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":464,"Value":929,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":20082000,"Value":67387000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":21,"Value":43,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9084000,"Value":16304000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":2246,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":464,"Executor Deserialize CPU Time":20082000,"Executor Run Time":21,"Executor CPU Time":9084000,"Result Size":1123,"JVM GC Time":18,"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,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":468,"Value":1397,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29183000,"Value":96570000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":21,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5753000,"Value":22057000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1123,"Value":3369,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":54,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":468,"Executor Deserialize CPU Time":29183000,"Executor Run Time":21,"Executor CPU Time":5753000,"Result Size":1123,"JVM GC Time":18,"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,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617257,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":470,"Value":1867,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":233387000,"Value":329957000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":22,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6783000,"Value":28840000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1938,"Value":5307,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":18,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":470,"Executor Deserialize CPU Time":233387000,"Executor Run Time":22,"Executor CPU Time":6783000,"Result Size":1938,"JVM GC Time":18,"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,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":453,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1479335616690,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617319,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":453,"Value":539,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":94,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":453,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617326,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Value":983,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":123,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":444,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1479335616687,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617327,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":1434,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":145,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1479335616696,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617328,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":451,"Value":1885,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":167,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":451,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":450,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1479335616693,"Executor ID":"2","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":450,"Value":2335,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":22,"Value":189,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":450,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":22,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1479335616694,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":444,"Value":2779,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":218,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":444,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":442,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1479335616688,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617329,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":442,"Value":3221,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":247,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":442,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":2,"Attempt":1,"Launch Time":1479335617332,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":1903,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5136000,"Value":346556000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3673,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":958000,"Value":32856000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9159,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":10,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617370,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":1889,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3808000,"Value":341420000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3672,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1005000,"Value":31898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8196,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":9,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617369,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":1879,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3737000,"Value":337612000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3670,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1066000,"Value":30893000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7233,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":13,"Attempt":1,"Launch Time":1479335617334,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617368,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":1872,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3918000,"Value":333875000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":987000,"Value":29827000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":6270,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"bad exec","Stack Trace":[{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply$mcII$sp","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":26},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":409},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1757},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":1135},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$5","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1927},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":87},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":99},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":282},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1142},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":617},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: bad exec\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply$mcII$sp(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat $line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1757)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1135)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1927)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:99)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)\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","Accumulator Updates":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":0,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1479335616691,"Executor ID":"0","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617336,"Failed":true,"Killed":false,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Update":446,"Value":3667,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":29,"Value":276,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":0,"Executor Run Time":446,"Executor CPU Time":0,"Result Size":0,"JVM GC Time":29,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":6,"Attempt":1,"Launch Time":1479335617349,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1907,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3503000,"Value":350059000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3674,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1042000,"Value":33898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10122,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":13,"Attempt":1,"Launch Time":1479335617334,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617368,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":1872,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3918000,"Value":333875000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":987000,"Value":29827000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":6270,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3918000,"Executor Run Time":1,"Executor CPU Time":987000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":1,"Attempt":1,"Launch Time":1479335617368,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617379,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1911,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3579000,"Value":353638000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3675,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":996000,"Value":34894000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11085,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":9,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617369,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":1879,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3737000,"Value":337612000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3670,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1066000,"Value":30893000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":7233,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":3737000,"Executor Run Time":2,"Executor CPU Time":1066000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":14,"Attempt":1,"Launch Time":1479335617369,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1915,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3412000,"Value":357050000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3676,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1014000,"Value":35908000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12048,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":10,"Attempt":1,"Launch Time":1479335617333,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617370,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":1889,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3808000,"Value":341420000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3672,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1005000,"Value":31898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":8196,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":3808000,"Executor Run Time":2,"Executor CPU Time":1005000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":5,"Attempt":1,"Launch Time":1479335617370,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1918,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3482000,"Value":360532000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3678,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1142000,"Value":37050000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":13011,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":2,"Attempt":1,"Launch Time":1479335617332,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":1903,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5136000,"Value":346556000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3673,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":958000,"Value":32856000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":9159,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":5136000,"Executor Run Time":1,"Executor CPU Time":958000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":6,"Attempt":1,"Launch Time":1479335617349,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617371,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1907,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3503000,"Value":350059000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3674,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1042000,"Value":33898000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":10122,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3503000,"Executor Run Time":1,"Executor CPU Time":1042000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":1,"Attempt":1,"Launch Time":1479335617368,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617379,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1911,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3579000,"Value":353638000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3675,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":996000,"Value":34894000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":11085,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3579000,"Executor Run Time":1,"Executor CPU Time":996000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":14,"Attempt":1,"Launch Time":1479335617369,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":1915,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3412000,"Value":357050000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":1,"Value":3676,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1014000,"Value":35908000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":12048,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3412000,"Executor Run Time":1,"Executor CPU Time":1014000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":5,"Attempt":1,"Launch Time":1479335617370,"Executor ID":"3","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617380,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1918,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3482000,"Value":360532000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":3678,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":1142000,"Value":37050000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":963,"Value":13011,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3482000,"Executor Run Time":2,"Executor CPU Time":1142000,"Result Size":963,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1479335616657,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617470,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":714,"Value":2632,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":41300000,"Value":401832000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":18,"Value":3696,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":6640000,"Value":43690000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":14047,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":293,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":714,"Executor Deserialize CPU Time":41300000,"Executor Run Time":18,"Executor CPU Time":6640000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1479335616692,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617471,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":714,"Value":3346,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":43682000,"Value":445514000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":15,"Value":3711,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":9441000,"Value":53131000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":15083,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":310,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":714,"Executor Deserialize CPU Time":43682000,"Executor Run Time":15,"Executor CPU Time":9441000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1479335616695,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617471,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":691,"Value":4037,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54811000,"Value":500325000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3727,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":4571000,"Value":57702000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1036,"Value":16119,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":327,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":691,"Executor Deserialize CPU Time":54811000,"Executor Run Time":16,"Executor CPU Time":4571000,"Result Size":1036,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1479335616689,"Executor ID":"1","Host":"172.22.0.167","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1479335617473,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":716,"Value":4753,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":220235000,"Value":720560000,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":16,"Value":3743,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":5849000,"Value":63551000,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":1851,"Value":17970,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":17,"Value":344,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Update":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":716,"Executor Deserialize CPU Time":220235000,"Executor Run Time":16,"Executor CPU Time":5849000,"Result Size":1851,"JVM GC Time":17,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at <console>:26","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:1135)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:31)\n$line16.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:33)\n$line16.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:35)\n$line16.$read$$iw$$iw$$iw$$iw.<init>(<console>:37)\n$line16.$read$$iw$$iw$$iw.<init>(<console>:39)\n$line16.$read$$iw$$iw.<init>(<console>:41)\n$line16.$read$$iw.<init>(<console>:43)\n$line16.$read.<init>(<console>:45)\n$line16.$read$.<init>(<console>:49)\n$line16.$read$.<clinit>(<console>)\n$line16.$eval$.$print$lzycompute(<console>:7)\n$line16.$eval$.$print(<console>:6)\n$line16.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)","Submission Time":1479335616653,"Completion Time":1479335617476,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":3743,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":344,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":17970,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.updatedBlockStatuses","Value":[{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}},{"Block ID":"broadcast_0_piece0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":false,"Replication":1},"Memory Size":1150,"Disk Size":0}},{"Block ID":"broadcast_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Deserialized":true,"Replication":1},"Memory Size":1736,"Disk Size":0}}],"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":720560000,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":63551000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":4753,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1479335617480,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479335617478,"executorId":"2","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerExecutorBlacklisted","time":1479335617478,"executorId":"0","taskFailures":4} +{"Event":"org.apache.spark.scheduler.SparkListenerNodeBlacklisted","time":1479335617478,"hostId":"172.22.0.167","executorFailures":2} +{"Event":"SparkListenerApplicationEnd","Timestamp":1479335620587} 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 d3b79dd3e36e838e92f2d1e4275ffbe654b0c053..cd3959a2b1666a591c62315035d8c9e076f6ed68 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 @@ -143,7 +143,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "stage task list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/stages/0/0/taskList", - "rdd list storage json" -> "applications/local-1422981780767/storage/rdd" + "rdd list storage json" -> "applications/local-1422981780767/storage/rdd", + "executor node blacklisting" -> "applications/app-20161116163331-0000/executors", + "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors" // Todo: enable this test when logging the even of onBlockUpdated. See: SPARK-13845 // "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" ) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 6b314d2ae339f092c3cb2193e26b65d1a0f7ab52..ead695574d292c10e415e5e946c9aeded0aa392c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import org.mockito.Mockito.when +import org.mockito.Mockito.{verify, when} import org.scalatest.BeforeAndAfterEach import org.scalatest.mock.MockitoSugar @@ -31,6 +31,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M private val clock = new ManualClock(0) private var blacklist: BlacklistTracker = _ + private var listenerBusMock: LiveListenerBus = _ private var scheduler: TaskSchedulerImpl = _ private var conf: SparkConf = _ @@ -40,7 +41,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M scheduler = mockTaskSchedWithConf(conf) clock.setTime(0) - blacklist = new BlacklistTracker(conf, clock) + + listenerBusMock = mock[LiveListenerBus] + blacklist = new BlacklistTracker(listenerBusMock, conf, clock) } override def afterEach(): Unit = { @@ -112,6 +115,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) } else { assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post( + SparkListenerExecutorBlacklisted(0, "1", failuresUntilBlacklisted)) } } } @@ -147,6 +152,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // and it should be blacklisted for the entire application. blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", numFailures)) } else { // The task set failed, so we don't count these failures against the executor for other // stages. @@ -166,6 +172,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M assert(blacklist.nodeBlacklist() === Set()) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) // Fail 4 tasks in one task set on executor 2, so that executor gets blacklisted for the whole @@ -177,15 +184,21 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures) assert(blacklist.nodeBlacklist() === Set("hostA")) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 4)) // Advance the clock and then make sure hostA and executors 1 and 2 have been removed from the // blacklist. - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) + val timeout = blacklist.BLACKLIST_TIMEOUT_MILLIS + 1 + clock.advance(timeout) blacklist.applyBlacklistTimeout() assert(blacklist.nodeBlacklist() === Set()) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "2")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(timeout, "1")) + verify(listenerBusMock).post(SparkListenerNodeUnblacklisted(timeout, "hostA")) // Fail one more task, but executor isn't put back into blacklist since the count of failures // on that executor should have been reset to 0. @@ -212,7 +225,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M stageAttemptId = 0, taskSetBlacklist0.execToFailures) assert(blacklist.isExecutorBlacklisted("1")) - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 4)) + val t1 = blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + clock.advance(t1) // Now another executor gets spun up on that host, but it also dies. val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) @@ -227,22 +242,29 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M // We've now had two bad executors on the hostA, so we should blacklist the entire node. assert(blacklist.isExecutorBlacklisted("1")) assert(blacklist.isExecutorBlacklisted("2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t1, "2", 4)) assert(blacklist.isNodeBlacklisted("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(t1, "hostA", 2)) // Advance the clock so that executor 1 should no longer be explicitly blacklisted, but // everything else should still be blacklisted. - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1) + val t2 = blacklist.BLACKLIST_TIMEOUT_MILLIS / 2 + 1 + clock.advance(t2) blacklist.applyBlacklistTimeout() assert(!blacklist.isExecutorBlacklisted("1")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(t1 + t2, "1")) assert(blacklist.isExecutorBlacklisted("2")) assert(blacklist.isNodeBlacklisted("hostA")) // make sure we don't leak memory assert(!blacklist.executorIdToBlacklistStatus.contains("1")) assert(!blacklist.nodeToBlacklistedExecs("hostA").contains("1")) // Advance the timeout again so now hostA should be removed from the blacklist. - clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS / 2) + clock.advance(t1) blacklist.applyBlacklistTimeout() assert(!blacklist.nodeIdToBlacklistExpiryTime.contains("hostA")) + verify(listenerBusMock).post(SparkListenerNodeUnblacklisted(t1 + t2 + t1, "hostA")) + // Even though unblacklisting a node implicitly unblacklists all of its executors, + // there will be no SparkListenerExecutorUnblacklisted sent here. } test("task failures expire with time") { @@ -280,6 +302,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M failOneTaskInTaskSet(exec = "1") blacklist.applyBlacklistTimeout() assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t1, "1", 2)) assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) // Add failures on executor 3, make sure it gets put on the blacklist. @@ -289,12 +312,14 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M failOneTaskInTaskSet(exec = "3") blacklist.applyBlacklistTimeout() assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "3")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(t2, "3", 2)) assert(blacklist.nextExpiryTime === t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS) // Now we go past the timeout for executor 1, so it should be dropped from the blacklist. clock.setTime(t1 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) blacklist.applyBlacklistTimeout() assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("3")) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(clock.getTimeMillis(), "1")) assert(blacklist.nextExpiryTime === t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS) // Make sure that we update correctly when we go from having blacklisted executors to @@ -308,6 +333,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M clock.setTime(t2 + blacklist.BLACKLIST_TIMEOUT_MILLIS + 1) blacklist.applyBlacklistTimeout() assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set()) + verify(listenerBusMock).post(SparkListenerExecutorUnblacklisted(clock.getTimeMillis(), "3")) // we've got one task failure still, but we don't bother setting nextExpiryTime to it, to // avoid wasting time checking for expiry of individual task failures. assert(blacklist.nextExpiryTime === Long.MaxValue) @@ -349,6 +375,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1) blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 2)) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1) @@ -356,6 +383,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1) blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, taskSetBlacklist1.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 2)) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set()) // Finally, blacklist another executor on the same node as the original blacklisted executor, @@ -365,7 +393,9 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "3", index = 1) blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures) assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2", "3")) + verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "3", 2)) assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set("hostA")) + verify(listenerBusMock).post(SparkListenerNodeBlacklisted(0, "hostA", 2)) } test("blacklist still respects legacy configs") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 2f5b029a966f122e67fca82f6304fb5cabaa7e0e..ffb9fe461a4863f58cbb697a529d83e5245066e3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -428,7 +428,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val clock = new ManualClock // We don't directly use the application blacklist, but its presence triggers blacklisting // within the taskset. - val blacklistTrackerOpt = Some(new BlacklistTracker(conf, clock)) + val mockListenerBus = mock(classOf[LiveListenerBus]) + val blacklistTrackerOpt = Some(new BlacklistTracker(mockListenerBus, conf, clock)) val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) { 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 85da79180fd0b85cf151df42cf845676f3e3f4f7..9f76c74bce89e2bd5f55bbf8f4f93e6ae3cf554a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -82,6 +82,12 @@ class JsonProtocolSuite extends SparkFunSuite { val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") + val executorBlacklisted = SparkListenerExecutorBlacklisted(executorBlacklistedTime, "exec1", 22) + val executorUnblacklisted = + SparkListenerExecutorUnblacklisted(executorUnblacklistedTime, "exec1") + val nodeBlacklisted = SparkListenerNodeBlacklisted(nodeBlacklistedTime, "node1", 33) + val nodeUnblacklisted = + SparkListenerNodeUnblacklisted(nodeUnblacklistedTime, "node1") val executorMetricsUpdate = { // Use custom accum ID for determinism val accumUpdates = @@ -109,6 +115,10 @@ class JsonProtocolSuite extends SparkFunSuite { testEvent(applicationEnd, applicationEndJsonString) testEvent(executorAdded, executorAddedJsonString) testEvent(executorRemoved, executorRemovedJsonString) + testEvent(executorBlacklisted, executorBlacklistedJsonString) + testEvent(executorUnblacklisted, executorUnblacklistedJsonString) + testEvent(nodeBlacklisted, nodeBlacklistedJsonString) + testEvent(nodeUnblacklisted, nodeUnblacklistedJsonString) testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString) } @@ -432,6 +442,10 @@ private[spark] object JsonProtocolSuite extends Assertions { private val jobCompletionTime = 1421191296660L private val executorAddedTime = 1421458410000L private val executorRemovedTime = 1421458922000L + private val executorBlacklistedTime = 1421458932000L + private val executorUnblacklistedTime = 1421458942000L + private val nodeBlacklistedTime = 1421458952000L + private val nodeUnblacklistedTime = 1421458962000L private def testEvent(event: SparkListenerEvent, jsonString: String) { val actualJsonString = compact(render(JsonProtocol.sparkEventToJson(event))) @@ -1980,4 +1994,39 @@ private[spark] object JsonProtocolSuite extends Assertions { | ] |} """.stripMargin + + private val executorBlacklistedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorBlacklisted", + | "time" : ${executorBlacklistedTime}, + | "executorId" : "exec1", + | "taskFailures" : 22 + |} + """.stripMargin + private val executorUnblacklistedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerExecutorUnblacklisted", + | "time" : ${executorUnblacklistedTime}, + | "executorId" : "exec1" + |} + """.stripMargin + private val nodeBlacklistedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeBlacklisted", + | "time" : ${nodeBlacklistedTime}, + | "hostId" : "node1", + | "executorFailures" : 33 + |} + """.stripMargin + private val nodeUnblacklistedJsonString = + s""" + |{ + | "Event" : "org.apache.spark.scheduler.SparkListenerNodeUnblacklisted", + | "time" : ${nodeUnblacklistedTime}, + | "hostId" : "node1" + |} + """.stripMargin } diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 6be1c72bc6cfb29355e16959663aa430152e772c..6d24434ccce7d814682f9d7ca17c910f12b0c5a3 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -73,6 +73,8 @@ logs .*dependency-reduced-pom.xml known_translations json_expectation +app-20161115172038-0000 +app-20161116163331-0000 local-1422981759269 local-1422981780767 local-1425081759269