You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ir...@apache.org on 2017/01/19 15:08:36 UTC

[1/4] spark git commit: [SPARK-16654][CORE] Add UI coverage for Application Level Blacklisting

Repository: spark
Updated Branches:
  refs/heads/master 064fadd2a -> 640f94233


http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
----------------------------------------------------------------------
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 d3b79dd..cd3959a 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"
   )

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
----------------------------------------------------------------------
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 6b314d2..ead6955 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") {

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
----------------------------------------------------------------------
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 2f5b029..ffb9fe4 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)
 
     {

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
----------------------------------------------------------------------
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 85da791..9f76c74 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
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/dev/.rat-excludes
----------------------------------------------------------------------
diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index 6be1c72..6d24434 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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[2/4] spark git commit: [SPARK-16654][CORE] Add UI coverage for Application Level Blacklisting

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/spark-events/app-20161116163331-0000
----------------------------------------------------------------------
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 0000000..7566c9f
--- /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.maxFailedEx
 ecutorsPerNode":"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.dat
 a.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.e
 ncoding":"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 C
 lasspath","/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/Idea
 Projects/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/tar
 get/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/scal
 a-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/j
 ars/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/spa
 rk/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/j
 ars/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 C
 lasspath","/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/ta
 rget/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","/U
 sers/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>(<consol
 e>: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,"Coun
 t 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,"Cou
 nt 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}}],"Inter
 nal":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 C
 lass":"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 Me
 mory":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 Byte
 s Spilled":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 C
 lass":"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":fal
 se,"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 C
 lass":"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 C
 lass":"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":fal
 se,"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 C
 lass":"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 C
 lass":"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":fal
 se,"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 C
 lass":"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 C
 lass":"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.m

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[3/4] spark git commit: [SPARK-16654][CORE] Add UI coverage for Application Level Blacklisting

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/spark-events/app-20161115172038-0000
----------------------------------------------------------------------
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 0000000..3af0451
--- /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.maxFailedEx
 ecutorsPerNode":"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 S
 pecification","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.enc
 oding":"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.comp
 ress=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 Class
 path","/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":"Sys
 tem 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","/Use
 rs/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/IdeaProj
 ects/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/a
 ssembly/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":"S
 ystem 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/jar
 s/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","/User
 s/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 Class
 path","/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/Idea
 Projects/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/targ
 et/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/ass
 embly/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/sca
 la-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/tar
 get/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>(<consol
 e>: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,"Cou
 nt 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,"Cou
 nt 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}}],"Inter
 nal":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 C
 lass":"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 C
 lass":"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 C
 lass":"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":fal
 se,"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 C
 lass":"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":fal
 se,"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 C
 lass":"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":fal
 se,"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 C
 lass":"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 C
 lass":"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 C
 lass":"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,"Na

<TRUNCATED>

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


[4/4] spark git commit: [SPARK-16654][CORE] Add UI coverage for Application Level Blacklisting

Posted by ir...@apache.org.
[SPARK-16654][CORE] Add UI coverage for Application Level Blacklisting

Builds on top of work in SPARK-8425 to update Application Level Blacklisting in the scheduler.

## What changes were proposed in this pull request?

Adds a UI to these patches by:
- defining new listener events for blacklisting and unblacklisting, nodes and executors;
- sending said events at the relevant points in BlacklistTracker;
- adding JSON (de)serialization code for these events;
- augmenting the Executors UI page to show which, and how many, executors are blacklisted;
- adding a unit test to make sure events are being fired;
- adding HistoryServerSuite coverage to verify that the SHS reads these events correctly.
- updates the Executor UI to show Blacklisted/Active/Dead as a tri-state in Executors Status

Updates .rat-excludes to pass tests.

username squito

## How was this patch tested?

./dev/run-tests
testOnly org.apache.spark.util.JsonProtocolSuite
testOnly org.apache.spark.scheduler.BlacklistTrackerSuite
testOnly org.apache.spark.deploy.history.HistoryServerSuite
https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh
![blacklist-20161219](https://cloud.githubusercontent.com/assets/1208477/21335321/9eda320a-c623-11e6-8b8c-9c912a73c276.jpg)

Author: Jos� Hiram Soltren <jo...@cloudera.com>

Closes #16346 from jsoltren/SPARK-16654-submit.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/640f9423
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/640f9423
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/640f9423

Branch: refs/heads/master
Commit: 640f942337e1ce87075195998bd051e19c4b50b9
Parents: 064fadd
Author: Jos� Hiram Soltren <jo...@cloudera.com>
Authored: Thu Jan 19 09:08:18 2017 -0600
Committer: Imran Rashid <ir...@cloudera.com>
Committed: Thu Jan 19 09:08:18 2017 -0600

----------------------------------------------------------------------
 .../org/apache/spark/SparkFirehoseListener.java |  20 ++++
 .../spark/ui/static/executorspage-template.html |   5 +
 .../org/apache/spark/ui/static/executorspage.js |  39 ++++--
 .../spark/scheduler/BlacklistTracker.scala      |  16 ++-
 .../spark/scheduler/EventLoggingListener.scala  |  16 +++
 .../apache/spark/scheduler/SparkListener.scala  |  54 +++++++++
 .../spark/scheduler/SparkListenerBus.scala      |   8 ++
 .../spark/scheduler/TaskSchedulerImpl.scala     |  10 +-
 .../org/apache/spark/status/api/v1/api.scala    |   1 +
 .../scala/org/apache/spark/ui/ToolTips.scala    |   3 +
 .../apache/spark/ui/exec/ExecutorsPage.scala    |   2 +
 .../org/apache/spark/ui/exec/ExecutorsTab.scala |  57 ++++++++-
 .../apache/spark/ui/jobs/ExecutorTable.scala    |   6 +
 .../scala/org/apache/spark/ui/jobs/UIData.scala |   2 +
 .../application_list_json_expectation.json      |  44 +++++--
 .../completed_app_list_json_expectation.json    |  44 +++++--
 .../executor_list_json_expectation.json         |   1 +
 .../executor_node_blacklisting_expectation.json | 118 +++++++++++++++++++
 ...blacklisting_unblacklisting_expectation.json | 118 +++++++++++++++++++
 .../limit_app_list_json_expectation.json        |  64 ++++------
 .../maxDate2_app_list_json_expectation.json     |   2 +-
 .../maxDate_app_list_json_expectation.json      |   4 +-
 .../minDate_app_list_json_expectation.json      |  40 ++++++-
 .../one_app_json_expectation.json               |   2 +-
 .../one_app_multi_attempt_json_expectation.json |   4 +-
 .../one_stage_attempt_json_expectation.json     |  80 ++++++-------
 .../one_stage_json_expectation.json             |  80 ++++++-------
 ...summary_w__custom_quantiles_expectation.json |   2 +-
 ...stage_with_accumulable_json_expectation.json | 100 ++++++++--------
 .../spark-events/app-20161115172038-0000        |  75 ++++++++++++
 .../spark-events/app-20161116163331-0000        |  68 +++++++++++
 .../deploy/history/HistoryServerSuite.scala     |   4 +-
 .../spark/scheduler/BlacklistTrackerSuite.scala |  42 ++++++-
 .../spark/scheduler/TaskSetManagerSuite.scala   |   3 +-
 .../apache/spark/util/JsonProtocolSuite.scala   |  49 ++++++++
 dev/.rat-excludes                               |   2 +
 36 files changed, 950 insertions(+), 235 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/java/org/apache/spark/SparkFirehoseListener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java
index 97eed61..9fe97b4 100644
--- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java
+++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java
@@ -114,6 +114,26 @@ public class SparkFirehoseListener implements SparkListenerInterface {
     }
 
     @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);
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html
----------------------------------------------------------------------
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 64ea719..4e83d6d 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>

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
----------------------------------------------------------------------
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 fe5db6a..7dbfe32 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,

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala
----------------------------------------------------------------------
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 bf7a62e..77d5c97 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)
         }
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
----------------------------------------------------------------------
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 f39565e..af9bdef 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 = {}
 

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
----------------------------------------------------------------------
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 7618dfe..1b12af7 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -106,6 +106,28 @@ case class SparkListenerExecutorRemoved(time: Long, executorId: String, reason:
   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
 
 /**
@@ -239,6 +261,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.
    */
   def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit
@@ -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 = { }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
----------------------------------------------------------------------
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 471586a..3ff3633 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

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
----------------------------------------------------------------------
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 9a8e313..72ed55a 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
     }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
----------------------------------------------------------------------
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 44a929b..7d035b1 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])
 

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
----------------------------------------------------------------------
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 3cc5353..766cc65 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

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
----------------------------------------------------------------------
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 7953d77..2d1691e 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
     )

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
----------------------------------------------------------------------
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 678571f..8ae712f 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)
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
----------------------------------------------------------------------
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 9fb3f35..cd1b02a 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 =>

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
----------------------------------------------------------------------
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 9ce8542..371dad9 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

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
----------------------------------------------------------------------
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 cba44c8..10902ab 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
   } ]
 } ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
----------------------------------------------------------------------
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 cba44c8..10902ab 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
   } ]
 } ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json
----------------------------------------------------------------------
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 e7db674..6b9f29e 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" : { }
 } ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json
----------------------------------------------------------------------
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 0000000..5914a1c
--- /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"
+  }
+} ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json
----------------------------------------------------------------------
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 0000000..92e249c
--- /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"
+  }
+} ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
----------------------------------------------------------------------
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 9165f54..8820c71 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
   } ]
 } ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/maxDate2_app_list_json_expectation.json
----------------------------------------------------------------------
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 a525d61..c3fe4db 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
   } ]
 } ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/maxDate_app_list_json_expectation.json
----------------------------------------------------------------------
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 cc567f6..8281fa7 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
   } ]
 } ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
----------------------------------------------------------------------
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 c934a87..1930281 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
   } ]
 } ]

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/one_app_json_expectation.json
----------------------------------------------------------------------
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 f486d46..e8ed96d 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
   } ]
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/one_app_multi_attempt_json_expectation.json
----------------------------------------------------------------------
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 e63039f..88c6015 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
   } ]
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
----------------------------------------------------------------------
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 0084339..cdebb5f 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
         }
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
----------------------------------------------------------------------
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 63fe3b2..85dafcf 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
         }
       }

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json
----------------------------------------------------------------------
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 5dcbc89..0ed609d 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 ],

http://git-wip-us.apache.org/repos/asf/spark/blob/640f9423/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
----------------------------------------------------------------------
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 aaeef1f..c2ece6f 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,


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org