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 2015/05/05 14:26:01 UTC

[01/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Repository: spark
Updated Branches:
  refs/heads/master 51f462003 -> d49735800


http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/scala/org/apache/spark/JsonTestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/JsonTestUtils.scala b/core/src/test/scala/org/apache/spark/JsonTestUtils.scala
new file mode 100644
index 0000000..ba367cd
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/JsonTestUtils.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark
+
+import org.json4s._
+import org.json4s.jackson.JsonMethods
+
+trait JsonTestUtils {
+  def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) {
+    val Diff(c, a, d) = validateJson.diff(expectedJson)
+    val validatePretty = JsonMethods.pretty(validateJson)
+    val expectedPretty = JsonMethods.pretty(expectedJson)
+    val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty"
+    import org.scalactic.TripleEquals._
+    assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}")
+    assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}")
+    assert(d === JNothing, s"$errorMessage\nDeleted:\n${JsonMethods.pretty(d)}")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index b58d625..e04a792 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -28,9 +28,9 @@ import org.scalatest.FunSuite
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
 import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo}
 import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
-import org.apache.spark.{SecurityManager, SparkConf}
+import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf}
 
-class JsonProtocolSuite extends FunSuite {
+class JsonProtocolSuite extends FunSuite with JsonTestUtils {
 
   test("writeApplicationInfo") {
     val output = JsonProtocol.writeApplicationInfo(createAppInfo())
@@ -136,16 +136,6 @@ class JsonProtocolSuite extends FunSuite {
       case e: JsonParseException => fail("Invalid Json detected", e)
     }
   }
-
-  def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) {
-    val Diff(c, a, d) = validateJson diff expectedJson
-    val validatePretty = JsonMethods.pretty(validateJson)
-    val expectedPretty = JsonMethods.pretty(expectedJson)
-    val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty"
-    assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}")
-    assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}")
-    assert(d === JNothing, s"$errorMessage\nDelected:\n${JsonMethods.pretty(d)}")
-  }
 }
 
 object JsonConstants {

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/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 71ba9c1..0744b68 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
@@ -14,22 +14,161 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.spark.deploy.history
 
-import javax.servlet.http.HttpServletRequest
-
-import scala.collection.mutable
+import java.io.{File, FileInputStream, FileWriter, IOException}
+import java.net.{HttpURLConnection, URL}
+import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
 
-import org.apache.hadoop.fs.Path
-import org.mockito.Mockito.{when}
-import org.scalatest.FunSuite
-import org.scalatest.Matchers
+import org.apache.commons.io.{FileUtils, IOUtils}
+import org.mockito.Mockito.when
+import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
 import org.scalatest.mock.MockitoSugar
 
+import org.apache.spark.{JsonTestUtils, SecurityManager, SparkConf}
 import org.apache.spark.ui.SparkUI
 
-class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar {
+/**
+ * A collection of tests against the historyserver, including comparing responses from the json
+ * metrics api to a set of known "golden files".  If new endpoints / parameters are added,
+ * cases should be added to this test suite.  The expected outcomes can be genered by running
+ * the HistoryServerSuite.main.  Note that this will blindly generate new expectation files matching
+ * the current behavior -- the developer must verify that behavior is correct.
+ *
+ * Similarly, if the behavior is changed, HistoryServerSuite.main can be run to update the
+ * expectations.  However, in general this should be done with extreme caution, as the metrics
+ * are considered part of Spark's public api.
+ */
+class HistoryServerSuite extends FunSuite with BeforeAndAfter with Matchers with MockitoSugar
+  with JsonTestUtils {
+
+  private val logDir = new File("src/test/resources/spark-events")
+  private val expRoot = new File("src/test/resources/HistoryServerExpectations/")
+
+  private var provider: FsHistoryProvider = null
+  private var server: HistoryServer = null
+  private var port: Int = -1
+
+  def init(): Unit = {
+    val conf = new SparkConf()
+      .set("spark.history.fs.logDirectory", logDir.getAbsolutePath)
+      .set("spark.history.fs.updateInterval", "0")
+      .set("spark.testing", "true")
+    provider = new FsHistoryProvider(conf)
+    provider.checkForLogs()
+    val securityManager = new SecurityManager(conf)
+
+    server = new HistoryServer(conf, provider, securityManager, 18080)
+    server.initialize()
+    server.bind()
+    port = server.boundPort
+  }
+
+  def stop(): Unit = {
+    server.stop()
+  }
+
+  before {
+    init()
+  }
+
+  after{
+    stop()
+  }
+
+  val cases = Seq(
+    "application list json" -> "applications",
+    "completed app list json" -> "applications?status=completed",
+    "running app list json" -> "applications?status=running",
+    "minDate app list json" -> "applications?minDate=2015-02-10",
+    "maxDate app list json" -> "applications?maxDate=2015-02-10",
+    "maxDate2 app list json" -> "applications?maxDate=2015-02-03T10:42:40.000CST",
+    "one app json" -> "applications/local-1422981780767",
+    "one app multi-attempt json" -> "applications/local-1426533911241",
+    "job list json" -> "applications/local-1422981780767/jobs",
+    "job list from multi-attempt app json(1)" -> "applications/local-1426533911241/1/jobs",
+    "job list from multi-attempt app json(2)" -> "applications/local-1426533911241/2/jobs",
+    "one job json" -> "applications/local-1422981780767/jobs/0",
+    "succeeded job list json" -> "applications/local-1422981780767/jobs?status=succeeded",
+    "succeeded&failed job list json" ->
+      "applications/local-1422981780767/jobs?status=succeeded&status=failed",
+    "executor list json" -> "applications/local-1422981780767/executors",
+    "stage list json" -> "applications/local-1422981780767/stages",
+    "complete stage list json" -> "applications/local-1422981780767/stages?status=complete",
+    "failed stage list json" -> "applications/local-1422981780767/stages?status=failed",
+    "one stage json" -> "applications/local-1422981780767/stages/1",
+    "one stage attempt json" -> "applications/local-1422981780767/stages/1/0",
+
+    "stage task summary" -> "applications/local-1427397477963/stages/20/0/taskSummary",
+    "stage task summary w/ custom quantiles" ->
+      "applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99",
+
+    "stage task list" -> "applications/local-1427397477963/stages/20/0/taskList",
+    "stage task list w/ offset & length" ->
+      "applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50",
+    "stage task list w/ sortBy" ->
+      "applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME",
+    "stage task list w/ sortBy short names: -runtime" ->
+      "applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime",
+    "stage task list w/ sortBy short names: runtime" ->
+      "applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime",
+
+    "stage list with accumulable json" -> "applications/local-1426533911241/1/stages",
+    "stage with accumulable json" -> "applications/local-1426533911241/1/stages/0/0",
+    "stage task list from multi-attempt app json(1)" ->
+      "applications/local-1426533911241/1/stages/0/0/taskList",
+    "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",
+    "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0"
+  )
+
+  // run a bunch of characterization tests -- just verify the behavior is the same as what is saved
+  // in the test resource folder
+  cases.foreach { case (name, path) =>
+    test(name) {
+      val (code, jsonOpt, errOpt) = getContentAndCode(path)
+      code should be (HttpServletResponse.SC_OK)
+      jsonOpt should be ('defined)
+      errOpt should be (None)
+      val json = jsonOpt.get
+      val exp = IOUtils.toString(new FileInputStream(
+        new File(expRoot, path + "/json_expectation")))
+      // compare the ASTs so formatting differences don't cause failures
+      import org.json4s._
+      import org.json4s.jackson.JsonMethods._
+      val jsonAst = parse(json)
+      val expAst = parse(exp)
+      assertValidDataInJson(jsonAst, expAst)
+    }
+  }
+
+  test("response codes on bad paths") {
+    val badAppId = getContentAndCode("applications/foobar")
+    badAppId._1 should be (HttpServletResponse.SC_NOT_FOUND)
+    badAppId._3 should be (Some("unknown app: foobar"))
+
+    val badStageId = getContentAndCode("applications/local-1422981780767/stages/12345")
+    badStageId._1 should be (HttpServletResponse.SC_NOT_FOUND)
+    badStageId._3 should be (Some("unknown stage: 12345"))
+
+    val badStageAttemptId = getContentAndCode("applications/local-1422981780767/stages/1/1")
+    badStageAttemptId._1 should be (HttpServletResponse.SC_NOT_FOUND)
+    badStageAttemptId._3 should be (Some("unknown attempt for stage 1.  Found attempts: [0]"))
+
+    val badStageId2 = getContentAndCode("applications/local-1422981780767/stages/flimflam")
+    badStageId2._1 should be (HttpServletResponse.SC_NOT_FOUND)
+    // will take some mucking w/ jersey to get a better error msg in this case
+
+    val badQuantiles = getContentAndCode(
+      "applications/local-1427397477963/stages/20/0/taskSummary?quantiles=foo,0.1")
+    badQuantiles._1 should be (HttpServletResponse.SC_BAD_REQUEST)
+    badQuantiles._3 should be (Some("Bad value for parameter \"quantiles\".  Expected a double, " +
+      "got \"foo\""))
+
+    getContentAndCode("foobar")._1 should be (HttpServletResponse.SC_NOT_FOUND)
+  }
 
   test("generate history page with relative links") {
     val historyServer = mock[HistoryServer]
@@ -54,4 +193,70 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar {
     } yield (attrs.toString)
     justHrefs should contain(link)
   }
+
+  def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = {
+    HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/json/v1/$path"))
+  }
+
+  def getUrl(path: String): String = {
+    HistoryServerSuite.getUrl(new URL(s"http://localhost:$port/json/v1/$path"))
+  }
+
+  def generateExpectation(path: String): Unit = {
+    val json = getUrl(path)
+    val dir = new File(expRoot, path)
+    dir.mkdirs()
+    val out = new FileWriter(new File(dir, "json_expectation"))
+    out.write(json)
+    out.close()
+  }
+}
+
+object HistoryServerSuite {
+  def main(args: Array[String]): Unit = {
+    // generate the "expected" results for the characterization tests.  Just blindly assume the
+    // current behavior is correct, and write out the returned json to the test/resource files
+
+    val suite = new HistoryServerSuite
+    FileUtils.deleteDirectory(suite.expRoot)
+    suite.expRoot.mkdirs()
+    try {
+      suite.init()
+      suite.cases.foreach { case (_, path) =>
+        suite.generateExpectation(path)
+      }
+    } finally {
+      suite.stop()
+    }
+  }
+
+  def getContentAndCode(url: URL): (Int, Option[String], Option[String]) = {
+    val connection = url.openConnection().asInstanceOf[HttpURLConnection]
+    connection.setRequestMethod("GET")
+    connection.connect()
+    val code = connection.getResponseCode()
+    val inString = try {
+      val in = Option(connection.getInputStream())
+      in.map{IOUtils.toString}
+    } catch {
+      case io: IOException => None
+    }
+    val errString = try {
+      val err = Option(connection.getErrorStream())
+      err.map{IOUtils.toString}
+    } catch {
+      case io: IOException => None
+    }
+    (code, inString, errString)
+  }
+
+  def getUrl(path: URL): String = {
+    val (code, resultOpt, error) = getContentAndCode(path)
+    if (code == 200) {
+      resultOpt.get
+    } else {
+      throw new RuntimeException(
+        "got code: " + code + " when getting " + path + " w/ error: " + error)
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala
new file mode 100644
index 0000000..5274df9
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamTest.scala
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import org.scalatest.{Matchers, FunSuite}
+
+class SimpleDateParamTest extends FunSuite with Matchers {
+
+  test("date parsing") {
+    new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be (1424474477190L)
+    new SimpleDateParam("2015-02-20T17:21:17.190CST").timestamp should be (1424474477190L)
+    new SimpleDateParam("2015-02-20").timestamp should be (1424390400000L)  // GMT
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index d53d7f3..117b2c3 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -18,11 +18,13 @@
 package org.apache.spark.ui
 
 import java.net.{HttpURLConnection, URL}
-import javax.servlet.http.HttpServletRequest
+import javax.servlet.http.{HttpServletResponse, HttpServletRequest}
 
 import scala.collection.JavaConversions._
 import scala.xml.Node
 
+import org.json4s._
+import org.json4s.jackson.JsonMethods
 import org.openqa.selenium.htmlunit.HtmlUnitDriver
 import org.openqa.selenium.{By, WebDriver}
 import org.scalatest._
@@ -33,8 +35,9 @@ import org.scalatest.time.SpanSugar._
 import org.apache.spark.LocalSparkContext._
 import org.apache.spark._
 import org.apache.spark.api.java.StorageLevels
+import org.apache.spark.deploy.history.HistoryServerSuite
 import org.apache.spark.shuffle.FetchFailedException
-
+import org.apache.spark.status.api.v1.{JacksonMessageWriter, StageStatus}
 
 /**
  * Selenium tests for the Spark Web UI.
@@ -42,6 +45,8 @@ import org.apache.spark.shuffle.FetchFailedException
 class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll {
 
   implicit var webDriver: WebDriver = _
+  implicit val formats = DefaultFormats
+
 
   override def beforeAll(): Unit = {
     webDriver = new HtmlUnitDriver
@@ -76,28 +81,42 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       val rdd = sc.parallelize(Seq(1, 2, 3))
       rdd.persist(StorageLevels.DISK_ONLY).count()
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (ui.appUIAddress.stripSuffix("/") + "/storage")
+        goToUi(ui, "/storage")
         val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq
         tableRowText should contain (StorageLevels.DISK_ONLY.description)
       }
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0")
+        goToUi(ui, "/storage/rdd/?id=0")
         val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq
         tableRowText should contain (StorageLevels.DISK_ONLY.description)
       }
 
+      val storageJson = getJson(ui, "storage/rdd")
+      storageJson.children.length should be (1)
+      (storageJson \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description)
+      val rddJson = getJson(ui, "storage/rdd/0")
+      (rddJson  \ "storageLevel").extract[String] should be (StorageLevels.DISK_ONLY.description)
+
       rdd.unpersist()
       rdd.persist(StorageLevels.MEMORY_ONLY).count()
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (ui.appUIAddress.stripSuffix("/") + "/storage")
+        goToUi(ui, "/storage")
         val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq
         tableRowText should contain (StorageLevels.MEMORY_ONLY.description)
       }
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0")
+        goToUi(ui, "/storage/rdd/?id=0")
         val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq
         tableRowText should contain (StorageLevels.MEMORY_ONLY.description)
       }
+
+      val updatedStorageJson = getJson(ui, "storage/rdd")
+      updatedStorageJson.children.length should be (1)
+      (updatedStorageJson \ "storageLevel").extract[String] should be (
+        StorageLevels.MEMORY_ONLY.description)
+      val updatedRddJson = getJson(ui, "storage/rdd/0")
+      (updatedRddJson  \ "storageLevel").extract[String] should be (
+        StorageLevels.MEMORY_ONLY.description)
     }
   }
 
@@ -108,10 +127,13 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
         sc.parallelize(1 to 10).map { x => throw new Exception()}.collect()
       }
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages")
+        goToUi(sc, "/stages")
         find(id("active")) should be(None)  // Since we hide empty tables
         find(id("failed")).get.text should be("Failed Stages (1)")
       }
+      val stageJson = getJson(sc.ui.get, "stages")
+      stageJson.children.length should be (1)
+      (stageJson \ "status").extract[String] should be (StageStatus.FAILED.name())
 
       // Regression test for SPARK-2105
       class NotSerializable
@@ -120,12 +142,15 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
         sc.parallelize(1 to 10).map { x => unserializableObject}.collect()
       }
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages")
+        goToUi(sc, "/stages")
         find(id("active")) should be(None)  // Since we hide empty tables
         // The failure occurs before the stage becomes active, hence we should still show only one
         // failed stage, not two:
         find(id("failed")).get.text should be("Failed Stages (1)")
       }
+
+      val updatedStageJson = getJson(sc.ui.get, "stages")
+      updatedStageJson should be (stageJson)
     }
   }
 
@@ -138,7 +163,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
     withSpark(newSparkContext(killEnabled = true)) { sc =>
       runSlowJob(sc)
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages")
+        goToUi(sc, "/stages")
         assert(hasKillLink)
       }
     }
@@ -146,7 +171,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
     withSpark(newSparkContext(killEnabled = false)) { sc =>
       runSlowJob(sc)
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages")
+        goToUi(sc, "/stages")
         assert(!hasKillLink)
       }
     }
@@ -157,7 +182,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       // If no job has been run in a job group, then "(Job Group)" should not appear in the header
       sc.parallelize(Seq(1, 2, 3)).count()
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs")
+        goToUi(sc, "/jobs")
         val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq
         tableHeaders should not contain "Job Id (Job Group)"
       }
@@ -165,10 +190,22 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       sc.setJobGroup("my-job-group", "my-job-group-description")
       sc.parallelize(Seq(1, 2, 3)).count()
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs")
+        goToUi(sc, "/jobs")
         val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq
         tableHeaders should contain ("Job Id (Job Group)")
       }
+
+      val jobJson = getJson(sc.ui.get, "jobs")
+      for {
+        job @ JObject(_) <- jobJson
+        JInt(jobId) <- job \ "jobId"
+        jobGroup = job \ "jobGroup"
+      } {
+        jobId.toInt match {
+          case 0 => jobGroup should be (JNothing)
+          case 1 => jobGroup should be (JString("my-job-group"))
+        }
+      }
     }
   }
 
@@ -195,7 +232,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       }
       mappedData.count()
       eventually(timeout(5 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs")
+        goToUi(sc, "/jobs")
         find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)")
         // Ideally, the following test would pass, but currently we overcount completed tasks
         // if task recomputations occur:
@@ -204,6 +241,32 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
         // of completed tasks may be higher:
         find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)")
       }
+      val jobJson = getJson(sc.ui.get, "jobs")
+      (jobJson \ "numTasks").extract[Int]should be (2)
+      (jobJson \ "numCompletedTasks").extract[Int] should be (3)
+      (jobJson \ "numFailedTasks").extract[Int] should be (1)
+      (jobJson \ "numCompletedStages").extract[Int] should be (2)
+      (jobJson \ "numFailedStages").extract[Int] should be (1)
+      val stageJson = getJson(sc.ui.get, "stages")
+
+      for {
+        stage @ JObject(_) <- stageJson
+        JString(status) <- stage \ "status"
+        JInt(stageId) <- stage \ "stageId"
+        JInt(attemptId) <- stage \ "attemptId"
+      } {
+        val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE
+        status should be (exp.name())
+      }
+
+      for {
+        stageId <- 0 to 1
+        attemptId <- 0 to 1
+      } {
+        val exp = if (attemptId == 0 && stageId == 1) StageStatus.FAILED else StageStatus.COMPLETE
+        val stageJson = getJson(sc.ui.get, s"stages/$stageId/$attemptId")
+        (stageJson \ "status").extract[String] should be (exp.name())
+      }
     }
   }
 
@@ -218,7 +281,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       // Start the job:
       rdd.countAsync()
       eventually(timeout(10 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0")
+        goToUi(sc, "/jobs/job/?id=0")
         find(id("active")).get.text should be ("Active Stages (1)")
         find(id("pending")).get.text should be ("Pending Stages (2)")
         // Essentially, we want to check that none of the stage rows show
@@ -244,7 +307,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       rdd.count()
       rdd.count()
       eventually(timeout(10 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs")
+        goToUi(sc, "/jobs")
         // The completed jobs table should have two rows. The first row will be the most recent job:
         val firstRow = find(cssSelector("tbody tr")).get.underlying
         val firstRowColumns = firstRow.findElements(By.tagName("td"))
@@ -271,7 +334,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       rdd.count()
       rdd.count()
       eventually(timeout(10 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1")
+        goToUi(sc, "/jobs/job/?id=1")
         find(id("pending")) should be (None)
         find(id("active")) should be (None)
         find(id("failed")) should be (None)
@@ -299,7 +362,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       rdd.count()
       rdd.count()
       eventually(timeout(10 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs")
+        goToUi(sc, "/jobs")
         findAll(cssSelector("tbody tr a")).foreach { link =>
           link.text.toLowerCase should include ("count")
           link.text.toLowerCase should not include "unknown"
@@ -321,7 +384,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       }
       sparkUI.attachTab(newTab)
       eventually(timeout(10 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/"))
+        goToUi(sc, "")
         find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None)
         find(cssSelector("""ul li a[href*="stages"]""")) should not be(None)
         find(cssSelector("""ul li a[href*="storage"]""")) should not be(None)
@@ -330,12 +393,12 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       }
       eventually(timeout(10 seconds), interval(50 milliseconds)) {
         // check whether new page exists
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo")
+        goToUi(sc, "/foo")
         find(cssSelector("b")).get.text should include ("html magic")
       }
       sparkUI.detachTab(newTab)
       eventually(timeout(10 seconds), interval(50 milliseconds)) {
-        go to (sc.ui.get.appUIAddress.stripSuffix("/"))
+        goToUi(sc, "")
         find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None)
         find(cssSelector("""ul li a[href*="stages"]""")) should not be(None)
         find(cssSelector("""ul li a[href*="storage"]""")) should not be(None)
@@ -344,7 +407,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       }
       eventually(timeout(10 seconds), interval(50 milliseconds)) {
         // check new page not exist
-        go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo")
+        goToUi(sc, "/foo")
         find(cssSelector("b")) should be(None)
       }
     }
@@ -371,4 +434,163 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before
       }
     }
   }
+
+  test("stage & job retention") {
+    val conf = new SparkConf()
+      .setMaster("local")
+      .setAppName("test")
+      .set("spark.ui.enabled", "true")
+      .set("spark.ui.port", "0")
+      .set("spark.ui.retainedStages", "3")
+      .set("spark.ui.retainedJobs", "2")
+    val sc = new SparkContext(conf)
+    assert(sc.ui.isDefined)
+
+    withSpark(sc) { sc =>
+      // run a few jobs & stages ...
+      (0 until 5).foreach { idx =>
+        // NOTE: if we reverse the order, things don't really behave nicely
+        // we lose the stage for a job we keep, and then the job doesn't know
+        // about its last stage
+        sc.parallelize(idx to (idx + 3)).map(identity).groupBy(identity).map(identity)
+          .groupBy(identity).count()
+        sc.parallelize(idx to (idx + 3)).collect()
+      }
+
+      val expJobInfo = Seq(
+        ("9", "collect"),
+        ("8", "count")
+      )
+
+      eventually(timeout(1 second), interval(50 milliseconds)) {
+        goToUi(sc, "/jobs")
+        // The completed jobs table should have two rows. The first row will be the most recent job:
+        find("completed-summary").get.text should be ("Completed Jobs: 10, only showing 2")
+        find("completed").get.text should be ("Completed Jobs (10, only showing 2)")
+        val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying}
+        rows.size should be (expJobInfo.size)
+        for {
+          (row, idx) <- rows.zipWithIndex
+          columns = row.findElements(By.tagName("td"))
+          id = columns(0).getText()
+          description = columns(1).getText()
+        } {
+          id should be (expJobInfo(idx)._1)
+          description should include (expJobInfo(idx)._2)
+        }
+      }
+
+      val jobsJson = getJson(sc.ui.get, "jobs")
+      jobsJson.children.size should be (expJobInfo.size)
+      for {
+        (job @ JObject(_),idx) <- jobsJson.children.zipWithIndex
+        id = (job \ "jobId").extract[String]
+        name = (job \ "name").extract[String]
+      } {
+        withClue(s"idx = $idx; id = $id; name = ${name.substring(0,20)}") {
+          id should be (expJobInfo(idx)._1)
+          name should include (expJobInfo(idx)._2)
+        }
+      }
+
+      // what about when we query for a job that did exist, but has been cleared?
+      goToUi(sc, "/jobs/job/?id=7")
+      find("no-info").get.text should be ("No information to display for job 7")
+
+      val badJob = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get, "jobs/7"))
+      badJob._1 should be (HttpServletResponse.SC_NOT_FOUND)
+      badJob._2 should be (None)
+      badJob._3 should be (Some("unknown job: 7"))
+
+      val expStageInfo = Seq(
+        ("19", "collect"),
+        ("18", "count"),
+        ("17", "groupBy")
+      )
+
+      eventually(timeout(1 second), interval(50 milliseconds)) {
+        goToUi(sc, "/stages")
+        find("completed-summary").get.text should be ("Completed Stages: 20, only showing 3")
+        find("completed").get.text should be ("Completed Stages (20, only showing 3)")
+        val rows = findAll(cssSelector("tbody tr")).toIndexedSeq.map{_.underlying}
+        rows.size should be (3)
+        for {
+          (row, idx) <- rows.zipWithIndex
+          columns = row.findElements(By.tagName("td"))
+          id = columns(0).getText()
+          description = columns(1).getText()
+        } {
+          id should be (expStageInfo(idx)._1)
+          description should include (expStageInfo(idx)._2)
+        }
+      }
+
+      val stagesJson = getJson(sc.ui.get, "stages")
+      stagesJson.children.size should be (3)
+      for {
+        (stage @ JObject(_), idx) <- stagesJson.children.zipWithIndex
+        id = (stage \ "stageId").extract[String]
+        name = (stage \ "name").extract[String]
+      } {
+        id should be (expStageInfo(idx)._1)
+        name should include (expStageInfo(idx)._2)
+      }
+
+      // nonexistent stage
+
+      goToUi(sc, "/stages/stage/?id=12&attempt=0")
+      find("no-info").get.text should be ("No information to display for Stage 12 (Attempt 0)")
+      val badStage = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/12/0"))
+      badStage._1 should be (HttpServletResponse.SC_NOT_FOUND)
+      badStage._2 should be (None)
+      badStage._3 should be (Some("unknown stage: 12"))
+
+      val badAttempt = HistoryServerSuite.getContentAndCode(jsonUrl(sc.ui.get,"stages/19/15"))
+      badAttempt._1 should be (HttpServletResponse.SC_NOT_FOUND)
+      badAttempt._2 should be (None)
+      badAttempt._3 should be (Some("unknown attempt for stage 19.  Found attempts: [0]"))
+
+      val badStageAttemptList = HistoryServerSuite.getContentAndCode(
+        jsonUrl(sc.ui.get, "stages/12"))
+      badStageAttemptList._1 should be (HttpServletResponse.SC_NOT_FOUND)
+      badStageAttemptList._2 should be (None)
+      badStageAttemptList._3 should be (Some("unknown stage: 12"))
+    }
+  }
+
+  test("live UI json application list") {
+    withSpark(newSparkContext()) { sc =>
+      val appListRawJson = HistoryServerSuite.getUrl(new URL(
+        sc.ui.get.appUIAddress + "/json/v1/applications"))
+      val appListJsonAst = JsonMethods.parse(appListRawJson)
+      appListJsonAst.children.length should be (1)
+      val attempts = (appListJsonAst \ "attempts").children
+      attempts.size should be (1)
+      (attempts(0) \ "completed").extract[Boolean] should be (false)
+      parseDate(attempts(0) \ "startTime") should be (sc.startTime)
+      parseDate(attempts(0) \ "endTime") should be (-1)
+      val oneAppJsonAst = getJson(sc.ui.get, "")
+      oneAppJsonAst should be (appListJsonAst.children(0))
+    }
+  }
+
+  def goToUi(sc: SparkContext, path: String): Unit = {
+    goToUi(sc.ui.get, path)
+  }
+
+  def goToUi(ui: SparkUI, path: String): Unit = {
+    go to (ui.appUIAddress.stripSuffix("/") + path)
+  }
+
+  def parseDate(json: JValue): Long = {
+    JacksonMessageWriter.makeISODateFormat.parse(json.extract[String]).getTime
+  }
+
+  def getJson(ui: SparkUI, path: String): JValue = {
+    JsonMethods.parse(HistoryServerSuite.getUrl(jsonUrl(ui, path)))
+  }
+
+  def jsonUrl(ui: SparkUI, path: String): URL = {
+    new URL(ui.appUIAddress + "/json/v1/applications/test/" + path)
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/docs/monitoring.md
----------------------------------------------------------------------
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 8a85928..1e0fc15 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -174,6 +174,80 @@ making it easy to identify slow tasks, data skew, etc.
 
 Note that the history server only displays completed Spark jobs. One way to signal the completion of a Spark job is to stop the Spark Context explicitly (`sc.stop()`), or in Python using the `with SparkContext() as sc:` to handle the Spark Context setup and tear down, and still show the job history on the UI.
 
+## REST API
+
+In addition to viewing the metrics in the UI, they are also available as JSON.  This gives developers
+an easy way to create new visualizations and monitoring tools for Spark.  The JSON is available for
+both running applications, and in the history server.  The endpoints are mounted at `/json/v1`.  Eg.,
+for the history server, they would typically be accessible at `http://<server-url>:18080/json/v1`, and
+for a running application, at `http://localhost:4040/json/v1`.
+
+<table class="table">
+  <tr><th>Endpoint</th><th>Meaning</th></tr>
+  <tr>
+    <td><code>/applications</code></td>
+    <td>A list of all applications</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/jobs</code></td>
+    <td>A list of all jobs for a given application</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/jobs/[job-id]</code></td>
+    <td>Details for the given job</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/stages</code></td>
+    <td>A list of all stages for a given application</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/stages/[stage-id]</code></td>
+    <td>A list of all attempts for the given stage</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/stages/[stage-id]/[stage-attempt-id]</code></td>
+    <td>Details for the given stage attempt</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskSummary</code></td>
+    <td>Summary metrics of all tasks in the given stage attempt</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/stages/[stage-id]/[stage-attempt-id]/taskList</code></td>
+    <td>A list of all tasks for the given stage attempt</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/executors</code></td>
+    <td>A list of all executors for the given application</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/storage/rdd</code></td>
+    <td>A list of stored RDDs for the given application</td>
+  </tr>
+  <tr>
+    <td><code>/applications/[app-id]/storage/rdd/[rdd-id]</code></td>
+    <td>Details for the storage status of a given RDD</td>
+  </tr>
+</table>
+
+When running on Yarn, each application has multiple attempts, so `[app-id]` is actually
+`[app-id]/[attempt-id]` in all cases.
+
+These endpoints have been strongly versioned to make it easier to develop applications on top.
+ In particular, Spark guarantees:
+
+* Endpoints will never be removed from one version
+* Individual fields will never be removed for any given endpoint
+* New endpoints may be added
+* New fields may be added to existing endpoints
+* New versions of the api may be added in the future at a separate endpoint (eg., `json/v2`).  New versions are *not* required to be backwards compatible.
+* Api versions may be dropped, but only after at least one minor release of co-existing with a new api version
+
+Note that even when examining the UI of a running applications, the `applications/[app-id]` portion is
+still required, though there is only one application available.  Eg. to see the list of jobs for the
+running app, you would go to `http://localhost:4040/json/v1/applications/[app-id]/jobs`.  This is to
+keep the paths consistent in both modes.
+
 # Metrics
 
 Spark has a configurable metrics system based on the 

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 12ad7c5..94cf283 100644
--- a/pom.xml
+++ b/pom.xml
@@ -624,6 +624,18 @@
         </exclusions>
       </dependency>
       <dependency>
+        <groupId>com.sun.jersey</groupId>
+        <artifactId>jersey-server</artifactId>
+        <version>1.9</version>
+        <scope>${hadoop.deps.scope}</scope>
+      </dependency>
+      <dependency>
+        <groupId>com.sun.jersey</groupId>
+        <artifactId>jersey-core</artifactId>
+        <version>1.9</version>
+        <scope>${hadoop.deps.scope}</scope>
+      </dependency>
+      <dependency>
         <groupId>org.scala-lang</groupId>
         <artifactId>scala-compiler</artifactId>
         <version>${scala.version}</version>


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


[03/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1
new file mode 100755
index 0000000..e704328
--- /dev/null
+++ b/core/src/test/resources/spark-events/local-1426633911242/EVENT_LOG_1
@@ -0,0 +1,24 @@
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"<driver>","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426633911361}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426633911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/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/irashid","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_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/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.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Li
 brary/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chica
 go","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/
 Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426633910242,"User":"irashid","App Attempt ID":"2"}
+{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$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:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\ns
 un.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"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":1426633936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"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":1426633936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"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":1426633936524,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\ns
 un.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426633936103,"Completion Time":1426633936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}}
+{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerApplicationEnd","Timestamp":1426633945177}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1426633911242/SPARK_VERSION_1.2.0
new file mode 100755
index 0000000..e69de29


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


[10/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
[SPARK-3454] separate json endpoints for data in the UI

Exposes data available in the UI as json over http.  Key points:

* new endpoints, handled independently of existing XyzPage classes.  Root entrypoint is `JsonRootResource`
* Uses jersey + jackson for routing & converting POJOs into json
* tests against known results in `HistoryServerSuite`
* also fixes some minor issues w/ the UI -- synchronizing on access to `StorageListener` & `StorageStatusListener`, and fixing some inconsistencies w/ the way we handle retained jobs & stages.

Author: Imran Rashid <ir...@cloudera.com>

Closes #4435 from squito/SPARK-3454 and squashes the following commits:

da1e35f [Imran Rashid] typos etc.
5e78b4f [Imran Rashid] fix rendering problems
5ae02ad [Imran Rashid] Merge branch 'master' into SPARK-3454
f016182 [Imran Rashid] change all constructors json-pojo class constructors to be private[spark] to protect us from mima-false-positives if we add fields
3347b72 [Imran Rashid] mark EnumUtil as @Private
ec140a2 [Imran Rashid] create @Private
cc1febf [Imran Rashid] add docs on the metrics-as-json api
cbaf287 [Imran Rashid] Merge branch 'master' into SPARK-3454
56db31e [Imran Rashid] update tests for mulit-attempt
7f3bc4e [Imran Rashid] Revert "add sbt-revolved plugin, to make it easier to start & stop http servers in sbt"
67008b4 [Imran Rashid] rats
9e51400 [Imran Rashid] style
c9bae1c [Imran Rashid] handle multiple attempts per app
b87cd63 [Imran Rashid] add sbt-revolved plugin, to make it easier to start & stop http servers in sbt
188762c [Imran Rashid] multi-attempt
2af11e5 [Imran Rashid] Merge branch 'master' into SPARK-3454
befff0c [Imran Rashid] review feedback
14ac3ed [Imran Rashid] jersey-core needs to be explicit; move version & scope to parent pom.xml
f90680e [Imran Rashid] Merge branch 'master' into SPARK-3454
dc8a7fe [Imran Rashid] style, fix errant comments
acb7ef6 [Imran Rashid] fix indentation
7bf1811 [Imran Rashid] move MetricHelper so mima doesnt think its exposed; comments
9d889d6 [Imran Rashid] undo some unnecessary changes
f48a7b0 [Imran Rashid] docs
52bbae8 [Imran Rashid] StorageListener & StorageStatusListener needs to synchronize internally to be thread-safe
31c79ce [Imran Rashid] asm no longer needed for SPARK_PREPEND_CLASSES
b2f8b91 [Imran Rashid] @DeveloperApi
2e19be2 [Imran Rashid] lazily convert ApplicationInfo to avoid memory overhead
ba3d9d2 [Imran Rashid] upper case enums
39ac29c [Imran Rashid] move EnumUtil
d2bde77 [Imran Rashid] update error handling & scoping
4a234d3 [Imran Rashid] avoid jersey-media-json-jackson b/c of potential version conflicts
a157a2f [Imran Rashid] style
7bd4d15 [Imran Rashid] delete security test, since it doesnt do anything
a325563 [Imran Rashid] style
a9c5cf1 [Imran Rashid] undo changes superceeded by master
0c6f968 [Imran Rashid] update deps
1ed0d07 [Imran Rashid] Merge branch 'master' into SPARK-3454
4c92af6 [Imran Rashid] style
f2e63ad [Imran Rashid] Merge branch 'master' into SPARK-3454
c22b11f [Imran Rashid] fix compile error
9ea682c [Imran Rashid] go back to good ol' java enums
cf86175 [Imran Rashid] style
d493b38 [Imran Rashid] Merge branch 'master' into SPARK-3454
f05ae89 [Imran Rashid] add in ExecutorSummaryInfo for MiMa :(
101a698 [Imran Rashid] style
d2ef58d [Imran Rashid] revert changes that had HistoryServer refresh the application listing more often
b136e39b [Imran Rashid] Revert "add sbt-revolved plugin, to make it easier to start & stop http servers in sbt"
e031719 [Imran Rashid] fixes from review
1f53a66 [Imran Rashid] style
b4a7863 [Imran Rashid] fix compile error
2c8b7ee [Imran Rashid] rats
1578a4a [Imran Rashid] doc
674f8dc [Imran Rashid] more explicit about total numbers of jobs & stages vs. number retained
9922be0 [Imran Rashid] Merge branch 'master' into stage_distributions
f5a5196 [Imran Rashid] undo removal of renderJson from MasterPage, since there is no substitute yet
db61211 [Imran Rashid] get JobProgressListener directly from UI
fdfc181 [Imran Rashid] stage/taskList
63eb4a6 [Imran Rashid] tests for taskSummary
ad27de8 [Imran Rashid] error handling on quantile values
b2efcaf [Imran Rashid] cleanup, combine stage-related paths into one resource
aaba896 [Imran Rashid] wire up task summary
a4b1397 [Imran Rashid] stage metric distributions
e48ba32 [Imran Rashid] rename
eaf3bbb [Imran Rashid] style
25cd894 [Imran Rashid] if only given day, assume GMT
51eaedb [Imran Rashid] more visibility fixes
9f28b7e [Imran Rashid] ack, more cleanup
99764e1 [Imran Rashid] Merge branch 'SPARK-3454_w_jersey' into SPARK-3454
a61a43c [Imran Rashid] oops, remove accidental checkin
a066055 [Imran Rashid] set visibility on a lot of classes
1f361c8 [Imran Rashid] update rat-excludes
0be5120 [Imran Rashid] Merge branch 'master' into SPARK-3454_w_jersey
2382bef [Imran Rashid] switch to using new "enum"
fef6605 [Imran Rashid] some utils for working w/ new "enum" format
dbfc7bf [Imran Rashid] style
b86bcb0 [Imran Rashid] update test to look at one stage attempt
5f9df24 [Imran Rashid] style
7fd156a [Imran Rashid] refactor jsonDiff to avoid code duplication
73f1378 [Imran Rashid] test json; also add test cases for cleaned stages & jobs
97d411f [Imran Rashid] json endpoint for one job
0c96147 [Imran Rashid] better error msgs for bad stageId vs bad attemptId
dddbd29 [Imran Rashid] stages have attempt; jobs are sorted; resource for all attempts for one stage
190c17a [Imran Rashid] StagePage should distinguish no task data, from unknown stage
84cd497 [Imran Rashid] AllJobsPage should still report correct completed & failed job count, even if some have been cleaned, to make it consistent w/ AllStagesPage
36e4062 [Imran Rashid] SparkUI needs to know about startTime, so it can list its own applicationInfo
b4c75ed [Imran Rashid] fix merge conflicts; need to widen visibility in a few cases
e91750a [Imran Rashid] Merge branch 'master' into SPARK-3454_w_jersey
56d2fc7 [Imran Rashid] jersey needs asm for SPARK_PREPEND_CLASSES to work
f7df095 [Imran Rashid] add test for accumulables, and discover that I need update after all
9c0c125 [Imran Rashid] add accumulableInfo
00e9cc5 [Imran Rashid] more style
3377e61 [Imran Rashid] scaladoc
d05f7a9 [Imran Rashid] dont use case classes for status api POJOs, since they have binary compatibility issues
654cecf [Imran Rashid] move all the status api POJOs to one file
b86e2b0 [Imran Rashid] style
18a8c45 [Imran Rashid] Merge branch 'master' into SPARK-3454_w_jersey
5598f19 [Imran Rashid] delete some unnecessary code, more to go
56edce0 [Imran Rashid] style
017c755 [Imran Rashid] add in metrics now available
1b78cb7 [Imran Rashid] fix some import ordering
0dc3ea7 [Imran Rashid] if app isnt found, reload apps from FS before giving up
c7d884f [Imran Rashid] fix merge conflicts
0c12b50 [Imran Rashid] Merge branch 'master' into SPARK-3454_w_jersey
b6a96a8 [Imran Rashid] compare json by AST, not string
cd37845 [Imran Rashid] switch to using java.util.Dates for times
a4ab5aa [Imran Rashid] add in explicit dependency on jersey 1.9 -- maven wasn't happy before this
4fdc39f [Imran Rashid] refactor case insensitive enum parsing
cba1ef6 [Imran Rashid] add security (maybe?) for metrics json
f0264a7 [Imran Rashid] switch to using jersey for metrics json
bceb3a9 [Imran Rashid] set http response code on error, some testing
e0356b6 [Imran Rashid] put new test expectation files in rat excludes (is this OK?)
b252e7a [Imran Rashid] small cleanup of accidental changes
d1a8c92 [Imran Rashid] add sbt-revolved plugin, to make it easier to start & stop http servers in sbt
4b398d0 [Imran Rashid] expose UI data as json in new endpoints


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

Branch: refs/heads/master
Commit: d49735800db27239c11478aac4b0f2ec9df91a3f
Parents: 51f4620
Author: Imran Rashid <ir...@cloudera.com>
Authored: Tue May 5 07:25:40 2015 -0500
Committer: Imran Rashid <ir...@cloudera.com>
Committed: Tue May 5 07:25:40 2015 -0500

----------------------------------------------------------------------
 .rat-excludes                                   |     7 +
 core/pom.xml                                    |     8 +
 .../org/apache/spark/JobExecutionStatus.java    |     8 +-
 .../spark/status/api/v1/ApplicationStatus.java  |    30 +
 .../apache/spark/status/api/v1/StageStatus.java |    31 +
 .../apache/spark/status/api/v1/TaskSorting.java |    48 +
 .../java/org/apache/spark/util/EnumUtil.java    |    38 +
 .../scala/org/apache/spark/SparkContext.scala   |     2 +-
 .../org/apache/spark/annotation/Private.java    |    41 +
 .../history/ApplicationHistoryProvider.scala    |     4 +-
 .../deploy/history/FsHistoryProvider.scala      |    14 +-
 .../spark/deploy/history/HistoryServer.scala    |    20 +-
 .../spark/deploy/master/ApplicationInfo.scala   |     2 +-
 .../org/apache/spark/deploy/master/Master.scala |    14 +-
 .../deploy/master/ui/ApplicationPage.scala      |    19 +-
 .../spark/deploy/master/ui/MasterPage.scala     |    12 +-
 .../spark/deploy/master/ui/MasterWebUI.scala    |    24 +-
 .../spark/status/api/v1/AllJobsResource.scala   |    98 +
 .../spark/status/api/v1/AllRDDResource.scala    |   104 +
 .../spark/status/api/v1/AllStagesResource.scala |   309 +
 .../status/api/v1/ApplicationListResource.scala |    94 +
 .../status/api/v1/ExecutorListResource.scala    |    36 +
 .../status/api/v1/JacksonMessageWriter.scala    |    93 +
 .../spark/status/api/v1/JsonRootResource.scala  |   255 +
 .../status/api/v1/OneApplicationResource.scala  |    31 +
 .../spark/status/api/v1/OneJobResource.scala    |    41 +
 .../spark/status/api/v1/OneRDDResource.scala    |    34 +
 .../spark/status/api/v1/OneStageResource.scala  |   150 +
 .../spark/status/api/v1/SecurityFilter.scala    |    38 +
 .../spark/status/api/v1/SimpleDateParam.scala   |    55 +
 .../org/apache/spark/status/api/v1/api.scala    |   228 +
 .../spark/storage/StorageStatusListener.scala   |     6 +-
 .../scala/org/apache/spark/ui/SparkUI.scala     |    49 +-
 .../main/scala/org/apache/spark/ui/WebUI.scala  |     8 +-
 .../apache/spark/ui/exec/ExecutorsPage.scala    |    17 +-
 .../org/apache/spark/ui/jobs/AllJobsPage.scala  |    14 +-
 .../apache/spark/ui/jobs/AllStagesPage.scala    |    12 +-
 .../org/apache/spark/ui/jobs/JobPage.scala      |     2 +-
 .../spark/ui/jobs/JobProgressListener.scala     |     4 +
 .../org/apache/spark/ui/jobs/PoolPage.scala     |     2 +-
 .../org/apache/spark/ui/jobs/StagePage.scala    |    19 +-
 .../org/apache/spark/ui/storage/RDDPage.scala   |    73 +-
 .../apache/spark/ui/storage/StoragePage.scala   |     2 +-
 .../apache/spark/ui/storage/StorageTab.scala    |     6 +-
 .../applications/json_expectation               |    53 +
 .../executors/json_expectation                  |    17 +
 .../local-1422981780767/jobs/0/json_expectation |    15 +
 .../local-1422981780767/jobs/json_expectation   |    43 +
 .../json_expectation                            |    43 +
 .../jobs?status=succeeded/json_expectation      |    29 +
 .../local-1422981780767/json_expectation        |    10 +
 .../stages/1/0/json_expectation                 |   270 +
 .../stages/1/json_expectation                   |   270 +
 .../local-1422981780767/stages/json_expectation |    89 +
 .../stages?status=complete/json_expectation     |    67 +
 .../stages?status=failed/json_expectation       |    23 +
 .../storage/rdd/0/json_expectation              |    64 +
 .../storage/rdd/json_expectation                |     9 +
 .../local-1426533911241/1/jobs/json_expectation |    15 +
 .../1/stages/0/0/json_expectation               |   242 +
 .../1/stages/0/0/taskList/json_expectation      |   193 +
 .../1/stages/json_expectation                   |    27 +
 .../local-1426533911241/2/jobs/json_expectation |    15 +
 .../2/stages/0/0/taskList/json_expectation      |   193 +
 .../local-1426533911241/json_expectation        |    17 +
 .../stages/20/0/taskList/json_expectation       |   481 +
 .../json_expectation                            |  1201 ++
 .../0/taskList?sortBy=-runtime/json_expectation |   481 +
 .../json_expectation                            |   481 +
 .../0/taskList?sortBy=runtime/json_expectation  |   481 +
 .../stages/20/0/taskSummary/json_expectation    |    15 +
 .../json_expectation                            |    15 +
 .../json_expectation                            |    10 +
 .../json_expectation                            |    19 +
 .../json_expectation                            |    35 +
 .../json_expectation                            |    53 +
 .../json_expectation                            |     1 +
 .../local-1422981759269/APPLICATION_COMPLETE    |     0
 .../local-1422981759269/EVENT_LOG_1             |    88 +
 .../local-1422981759269/SPARK_VERSION_1.2.0     |     0
 .../local-1422981780767/APPLICATION_COMPLETE    |     0
 .../local-1422981780767/EVENT_LOG_1             |    82 +
 .../local-1422981780767/SPARK_VERSION_1.2.0     |     0
 .../local-1425081759269/APPLICATION_COMPLETE    |     0
 .../local-1425081759269/EVENT_LOG_1             |    88 +
 .../local-1425081759269/SPARK_VERSION_1.2.0     |     0
 .../local-1426533911241/APPLICATION_COMPLETE    |     0
 .../local-1426533911241/EVENT_LOG_1             |    24 +
 .../local-1426533911241/SPARK_VERSION_1.2.0     |     0
 .../local-1426633911242/APPLICATION_COMPLETE    |     0
 .../local-1426633911242/EVENT_LOG_1             |    24 +
 .../local-1426633911242/SPARK_VERSION_1.2.0     |     0
 .../resources/spark-events/local-1427397477963  | 12083 +++++++++++++++++
 .../scala/org/apache/spark/JsonTestUtils.scala  |    34 +
 .../apache/spark/deploy/JsonProtocolSuite.scala |    14 +-
 .../deploy/history/HistoryServerSuite.scala     |   223 +-
 .../status/api/v1/SimpleDateParamTest.scala     |    29 +
 .../org/apache/spark/ui/UISeleniumSuite.scala   |   264 +-
 docs/monitoring.md                              |    74 +
 pom.xml                                         |    12 +
 100 files changed, 19946 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/.rat-excludes
----------------------------------------------------------------------
diff --git a/.rat-excludes b/.rat-excludes
index dccf2db..ac652ed 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -74,5 +74,12 @@ logs
 .*scalastyle-output.xml
 .*dependency-reduced-pom.xml
 known_translations
+json_expectation
+local-1422981759269/*
+local-1422981780767/*
+local-1425081759269/*
+local-1426533911241/*
+local-1426633911242/*
+local-1427397477963/*
 DESCRIPTION
 NAMESPACE

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 164a836..fc42f48 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -229,6 +229,14 @@
       <version>3.2.10</version>
     </dependency>
     <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-core</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.mesos</groupId>
       <artifactId>mesos</artifactId>
       <classifier>${mesos.classifier}</classifier>

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/java/org/apache/spark/JobExecutionStatus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/JobExecutionStatus.java b/core/src/main/java/org/apache/spark/JobExecutionStatus.java
index 6e16131..0287fb7 100644
--- a/core/src/main/java/org/apache/spark/JobExecutionStatus.java
+++ b/core/src/main/java/org/apache/spark/JobExecutionStatus.java
@@ -17,9 +17,15 @@
 
 package org.apache.spark;
 
+import org.apache.spark.util.EnumUtil;
+
 public enum JobExecutionStatus {
   RUNNING,
   SUCCEEDED,
   FAILED,
-  UNKNOWN
+  UNKNOWN;
+
+  public static JobExecutionStatus fromString(String str) {
+    return EnumUtil.parseIgnoreCase(JobExecutionStatus.class, str);
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java
new file mode 100644
index 0000000..8c7dcf7
--- /dev/null
+++ b/core/src/main/java/org/apache/spark/status/api/v1/ApplicationStatus.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.status.api.v1;
+
+import org.apache.spark.util.EnumUtil;
+
+public enum ApplicationStatus {
+  COMPLETED,
+  RUNNING;
+
+  public static ApplicationStatus fromString(String str) {
+    return EnumUtil.parseIgnoreCase(ApplicationStatus.class, str);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java
new file mode 100644
index 0000000..9dbb565
--- /dev/null
+++ b/core/src/main/java/org/apache/spark/status/api/v1/StageStatus.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.status.api.v1;
+
+import org.apache.spark.util.EnumUtil;
+
+public enum StageStatus {
+  ACTIVE,
+  COMPLETE,
+  FAILED,
+  PENDING;
+
+  public static StageStatus fromString(String str) {
+    return EnumUtil.parseIgnoreCase(StageStatus.class, str);
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java
new file mode 100644
index 0000000..f19ed01
--- /dev/null
+++ b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.status.api.v1;
+
+import org.apache.spark.util.EnumUtil;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public enum TaskSorting {
+  ID,
+  INCREASING_RUNTIME("runtime"),
+  DECREASING_RUNTIME("-runtime");
+
+  private final Set<String> alternateNames;
+  private TaskSorting(String... names) {
+    alternateNames = new HashSet<String>();
+    for (String n: names) {
+      alternateNames.add(n);
+    }
+  }
+
+  public static TaskSorting fromString(String str) {
+    String lower = str.toLowerCase();
+    for (TaskSorting t: values()) {
+      if (t.alternateNames.contains(lower)) {
+        return t;
+      }
+    }
+    return EnumUtil.parseIgnoreCase(TaskSorting.class, str);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/java/org/apache/spark/util/EnumUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/spark/util/EnumUtil.java b/core/src/main/java/org/apache/spark/util/EnumUtil.java
new file mode 100644
index 0000000..c40c7e7
--- /dev/null
+++ b/core/src/main/java/org/apache/spark/util/EnumUtil.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.util;
+
+import com.google.common.base.Joiner;
+import org.apache.spark.annotation.Private;
+
+@Private
+public class EnumUtil {
+  public static <E extends Enum<E>> E parseIgnoreCase(Class<E> clz, String str) {
+    E[] constants = clz.getEnumConstants();
+    if (str == null) {
+      return null;
+    }
+    for (E e : constants) {
+      if (e.name().equalsIgnoreCase(str)) {
+        return e;
+      }
+    }
+    throw new IllegalArgumentException(
+      String.format("Illegal type='%s'. Supported type values: %s",
+        str, Joiner.on(", ").join(constants)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index b98a54b..7ebee99 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -428,7 +428,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
     _ui =
       if (conf.getBoolean("spark.ui.enabled", true)) {
         Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener,
-          _env.securityManager,appName))
+          _env.securityManager,appName, startTime = startTime))
       } else {
         // For tests, do not enable the UI
         None

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/annotation/Private.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/annotation/Private.java b/core/src/main/scala/org/apache/spark/annotation/Private.java
new file mode 100644
index 0000000..9082fcf
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/annotation/Private.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.annotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * A class that is considered private to the internals of Spark -- there is a high-likelihood
+ * they will be changed in future versions of Spark.
+ *
+ * This should be used only when the standard Scala / Java means of protecting classes are
+ * insufficient.  In particular, Java has no equivalent of private[spark], so we use this annotation
+ * in its place.
+ *
+ * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first
+ * line of the comment must be ":: Private ::" with no trailing blank line. This is because
+ * of the known issue that Scaladoc displays only either the annotation or the comment, whichever
+ * comes first.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER,
+        ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE})
+public @interface Private {}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
index 6a5011a..298a820 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala
@@ -19,7 +19,7 @@ package org.apache.spark.deploy.history
 
 import org.apache.spark.ui.SparkUI
 
-private[history] case class ApplicationAttemptInfo(
+private[spark] case class ApplicationAttemptInfo(
     attemptId: Option[String],
     startTime: Long,
     endTime: Long,
@@ -27,7 +27,7 @@ private[history] case class ApplicationAttemptInfo(
     sparkUser: String,
     completed: Boolean = false)
 
-private[history] case class ApplicationHistoryInfo(
+private[spark] case class ApplicationHistoryInfo(
     id: String,
     name: String,
     attempts: List[ApplicationAttemptInfo])

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index 993763f..45c2be3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -17,23 +17,21 @@
 
 package org.apache.spark.deploy.history
 
-import java.io.{IOException, BufferedInputStream, FileNotFoundException, InputStream}
+import java.io.{BufferedInputStream, FileNotFoundException, IOException, InputStream}
 import java.util.concurrent.{ExecutorService, Executors, TimeUnit}
 
 import scala.collection.mutable
-import scala.concurrent.duration.Duration
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder
-
-import com.google.common.util.concurrent.MoreExecutors
-import org.apache.hadoop.fs.permission.AccessControlException
+import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder}
 import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hadoop.fs.permission.AccessControlException
+
+import org.apache.spark.{Logging, SecurityManager, SparkConf}
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.scheduler._
 import org.apache.spark.ui.SparkUI
 import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils}
-import org.apache.spark.{Logging, SecurityManager, SparkConf}
 
 /**
  * A class that provides application history from event logs stored in the file system.
@@ -151,7 +149,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
             val conf = this.conf.clone()
             val appSecManager = new SecurityManager(conf)
             SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId,
-              HistoryServer.getAttemptURI(appId, attempt.attemptId))
+              HistoryServer.getAttemptURI(appId, attempt.attemptId), attempt.startTime)
             // Do not call ui.bind() to avoid creating a new server for each application
           }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index 754c8e9..50522e6 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -25,6 +25,7 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
 
 import org.apache.spark.{Logging, SecurityManager, SparkConf}
 import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.status.api.v1.{ApplicationInfo, ApplicationsListResource, JsonRootResource, UIRoot}
 import org.apache.spark.ui.{SparkUI, UIUtils, WebUI}
 import org.apache.spark.ui.JettyUtils._
 import org.apache.spark.util.{SignalLogger, Utils}
@@ -45,7 +46,7 @@ class HistoryServer(
     provider: ApplicationHistoryProvider,
     securityManager: SecurityManager,
     port: Int)
-  extends WebUI(securityManager, port, conf) with Logging {
+  extends WebUI(securityManager, port, conf) with Logging with UIRoot {
 
   // How many applications to retain
   private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50)
@@ -56,7 +57,7 @@ class HistoryServer(
       require(parts.length == 1 || parts.length == 2, s"Invalid app key $key")
       val ui = provider
         .getAppUI(parts(0), if (parts.length > 1) Some(parts(1)) else None)
-        .getOrElse(throw new NoSuchElementException())
+        .getOrElse(throw new NoSuchElementException(s"no app with key $key"))
       attachSparkUI(ui)
       ui
     }
@@ -113,6 +114,10 @@ class HistoryServer(
     }
   }
 
+  def getSparkUI(appKey: String): Option[SparkUI] = {
+    Option(appCache.get(appKey))
+  }
+
   initialize()
 
   /**
@@ -123,6 +128,9 @@ class HistoryServer(
    */
   def initialize() {
     attachPage(new HistoryPage(this))
+
+    attachHandler(JsonRootResource.getJsonServlet(this))
+
     attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
 
     val contextHandler = new ServletContextHandler
@@ -160,7 +168,13 @@ class HistoryServer(
    *
    * @return List of all known applications.
    */
-  def getApplicationList(): Iterable[ApplicationHistoryInfo] = provider.getListing()
+  def getApplicationList(): Iterable[ApplicationHistoryInfo] = {
+    provider.getListing()
+  }
+
+  def getApplicationInfoList: Iterator[ApplicationInfo] = {
+    getApplicationList().iterator.map(ApplicationsListResource.appHistoryInfoToPublicAppInfo)
+  }
 
   /**
    * Returns the provider configuration to show in the listing page.

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
index f59d550..1620e95 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
@@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.deploy.ApplicationDescription
 import org.apache.spark.util.Utils
 
-private[deploy] class ApplicationInfo(
+private[spark] class ApplicationInfo(
     val startTime: Long,
     val id: String,
     val desc: ApplicationDescription,

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 0fac3cd..53e1903 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -754,9 +754,9 @@ private[master] class Master(
 
   /**
    * Rebuild a new SparkUI from the given application's event logs.
-   * Return whether this is successful.
+   * Return the UI if successful, else None
    */
-  private def rebuildSparkUI(app: ApplicationInfo): Boolean = {
+  private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = {
     val appName = app.desc.name
     val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found"
     try {
@@ -764,7 +764,7 @@ private[master] class Master(
         .getOrElse {
           // Event logging is not enabled for this application
           app.desc.appUiUrl = notFoundBasePath
-          return false
+          return None
         }
 
       val eventLogFilePrefix = EventLoggingListener.getLogPath(
@@ -787,7 +787,7 @@ private[master] class Master(
       val logInput = EventLoggingListener.openEventLog(new Path(eventLogFile), fs)
       val replayBus = new ReplayListenerBus()
       val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf),
-        appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}")
+        appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}", app.startTime)
       val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS)
       try {
         replayBus.replay(logInput, eventLogFile, maybeTruncated)
@@ -798,7 +798,7 @@ private[master] class Master(
       webUi.attachSparkUI(ui)
       // Application UI is successfully rebuilt, so link the Master UI to it
       app.desc.appUiUrl = ui.basePath
-      true
+      Some(ui)
     } catch {
       case fnf: FileNotFoundException =>
         // Event logging is enabled for this application, but no event logs are found
@@ -808,7 +808,7 @@ private[master] class Master(
         msg += " Did you specify the correct logging directory?"
         msg = URLEncoder.encode(msg, "UTF-8")
         app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title"
-        false
+        None
       case e: Exception =>
         // Relay exception message to application UI page
         val title = s"Application history load error (${app.id})"
@@ -817,7 +817,7 @@ private[master] class Master(
         logError(msg, e)
         msg = URLEncoder.encode(msg, "UTF-8")
         app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title"
-        false
+        None
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index 273f077..06e265f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -23,10 +23,8 @@ import scala.concurrent.Await
 import scala.xml.Node
 
 import akka.pattern.ask
-import org.json4s.JValue
-import org.json4s.JsonAST.JNothing
 
-import org.apache.spark.deploy.{ExecutorState, JsonProtocol}
+import org.apache.spark.deploy.ExecutorState
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
 import org.apache.spark.deploy.master.ExecutorDesc
 import org.apache.spark.ui.{UIUtils, WebUIPage}
@@ -38,21 +36,6 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
   private val timeout = parent.timeout
 
   /** Executor details for a particular application */
-  override def renderJson(request: HttpServletRequest): JValue = {
-    val appId = request.getParameter("appId")
-    val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
-    val state = Await.result(stateFuture, timeout)
-    val app = state.activeApps.find(_.id == appId).getOrElse({
-      state.completedApps.find(_.id == appId).getOrElse(null)
-    })
-    if (app == null) {
-      JNothing
-    } else {
-      JsonProtocol.writeApplicationInfo(app)
-    }
-  }
-
-  /** Executor details for a particular application */
   def render(request: HttpServletRequest): Seq[Node] = {
     val appId = request.getParameter("appId")
     val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index 1f2c3fd..7569276 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -35,10 +35,13 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
   private val master = parent.masterActorRef
   private val timeout = parent.timeout
 
-  override def renderJson(request: HttpServletRequest): JValue = {
+  def getMasterState: MasterStateResponse = {
     val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
-    val state = Await.result(stateFuture, timeout)
-    JsonProtocol.writeMasterState(state)
+    Await.result(stateFuture, timeout)
+  }
+
+  override def renderJson(request: HttpServletRequest): JValue = {
+    JsonProtocol.writeMasterState(getMasterState)
   }
 
   def handleAppKillRequest(request: HttpServletRequest): Unit = {
@@ -68,8 +71,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
 
   /** Index view listing applications and executors */
   def render(request: HttpServletRequest): Seq[Node] = {
-    val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
-    val state = Await.result(stateFuture, timeout)
+    val state = getMasterState
 
     val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", "Memory")
     val workers = state.workers.sortBy(_.id)

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index dea0a65..eb26e9f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -19,6 +19,7 @@ package org.apache.spark.deploy.master.ui
 
 import org.apache.spark.Logging
 import org.apache.spark.deploy.master.Master
+import org.apache.spark.status.api.v1.{ApplicationsListResource, ApplicationInfo, JsonRootResource, UIRoot}
 import org.apache.spark.ui.{SparkUI, WebUI}
 import org.apache.spark.ui.JettyUtils._
 import org.apache.spark.util.RpcUtils
@@ -28,12 +29,15 @@ import org.apache.spark.util.RpcUtils
  */
 private[master]
 class MasterWebUI(val master: Master, requestedPort: Int)
-  extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging {
+  extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging
+  with UIRoot {
 
   val masterActorRef = master.self
   val timeout = RpcUtils.askTimeout(master.conf)
   val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true)
 
+  val masterPage = new MasterPage(this)
+
   initialize()
 
   /** Initialize all components of the server. */
@@ -43,6 +47,7 @@ class MasterWebUI(val master: Master, requestedPort: Int)
     attachPage(new HistoryNotFoundPage(this))
     attachPage(masterPage)
     attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"))
+    attachHandler(JsonRootResource.getJsonServlet(this))
     attachHandler(createRedirectHandler(
       "/app/kill", "/", masterPage.handleAppKillRequest, httpMethods = Set("POST")))
     attachHandler(createRedirectHandler(
@@ -60,6 +65,23 @@ class MasterWebUI(val master: Master, requestedPort: Int)
     assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs")
     ui.getHandlers.foreach(detachHandler)
   }
+
+  def getApplicationInfoList: Iterator[ApplicationInfo] = {
+    val state = masterPage.getMasterState
+    val activeApps = state.activeApps.sortBy(_.startTime).reverse
+    val completedApps = state.completedApps.sortBy(_.endTime).reverse
+    activeApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, false) } ++
+      completedApps.iterator.map { ApplicationsListResource.convertApplicationInfo(_, true) }
+  }
+
+  def getSparkUI(appId: String): Option[SparkUI] = {
+    val state = masterPage.getMasterState
+    val activeApps = state.activeApps.sortBy(_.startTime).reverse
+    val completedApps = state.completedApps.sortBy(_.endTime).reverse
+    (activeApps ++ completedApps).find { _.id == appId }.flatMap {
+      master.rebuildSparkUI
+    }
+  }
 }
 
 private[master] object MasterWebUI {

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala
new file mode 100644
index 0000000..5783df5
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllJobsResource.scala
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.{Arrays, Date, List => JList}
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllJobsResource(ui: SparkUI) {
+
+  @GET
+  def jobsList(@QueryParam("status") statuses: JList[JobExecutionStatus]): Seq[JobData] = {
+    val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] =
+      AllJobsResource.getStatusToJobs(ui)
+    val adjStatuses: JList[JobExecutionStatus] = {
+      if (statuses.isEmpty) {
+        Arrays.asList(JobExecutionStatus.values(): _*)
+      } else {
+        statuses
+      }
+    }
+    val jobInfos = for {
+      (status, jobs) <- statusToJobs
+      job <- jobs if adjStatuses.contains(status)
+    } yield {
+      AllJobsResource.convertJobData(job, ui.jobProgressListener, false)
+    }
+    jobInfos.sortBy{- _.jobId}
+  }
+
+}
+
+private[v1] object AllJobsResource {
+
+  def getStatusToJobs(ui: SparkUI): Seq[(JobExecutionStatus, Seq[JobUIData])] = {
+    val statusToJobs = ui.jobProgressListener.synchronized {
+      Seq(
+        JobExecutionStatus.RUNNING -> ui.jobProgressListener.activeJobs.values.toSeq,
+        JobExecutionStatus.SUCCEEDED -> ui.jobProgressListener.completedJobs.toSeq,
+        JobExecutionStatus.FAILED -> ui.jobProgressListener.failedJobs.reverse.toSeq
+      )
+    }
+    statusToJobs
+  }
+
+  def convertJobData(
+      job: JobUIData,
+      listener: JobProgressListener,
+      includeStageDetails: Boolean): JobData = {
+    listener.synchronized {
+      val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max)
+      val lastStageData = lastStageInfo.flatMap { s =>
+        listener.stageIdToData.get((s.stageId, s.attemptId))
+      }
+      val lastStageName = lastStageInfo.map { _.name }.getOrElse("(Unknown Stage Name)")
+      val lastStageDescription = lastStageData.flatMap { _.description }
+      new JobData(
+        jobId = job.jobId,
+        name = lastStageName,
+        description = lastStageDescription,
+        submissionTime = job.submissionTime.map{new Date(_)},
+        completionTime = job.completionTime.map{new Date(_)},
+        stageIds = job.stageIds,
+        jobGroup = job.jobGroup,
+        status = job.status,
+        numTasks = job.numTasks,
+        numActiveTasks = job.numActiveTasks,
+        numCompletedTasks = job.numCompletedTasks,
+        numSkippedTasks = job.numCompletedTasks,
+        numFailedTasks = job.numFailedTasks,
+        numActiveStages = job.numActiveStages,
+        numCompletedStages = job.completedStageIndices.size,
+        numSkippedStages = job.numSkippedStages,
+        numFailedStages = job.numFailedStages
+      )
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala
new file mode 100644
index 0000000..645ede2
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.{GET, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.storage.{RDDInfo, StorageStatus, StorageUtils}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.storage.StorageListener
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllRDDResource(ui: SparkUI) {
+
+  @GET
+  def rddList(): Seq[RDDStorageInfo] = {
+    val storageStatusList = ui.storageListener.storageStatusList
+    val rddInfos = ui.storageListener.rddInfoList
+    rddInfos.map{rddInfo =>
+      AllRDDResource.getRDDStorageInfo(rddInfo.id, rddInfo, storageStatusList,
+        includeDetails = false)
+    }
+  }
+
+}
+
+private[spark] object AllRDDResource {
+
+  def getRDDStorageInfo(
+      rddId: Int,
+      listener: StorageListener,
+      includeDetails: Boolean): Option[RDDStorageInfo] = {
+    val storageStatusList = listener.storageStatusList
+    listener.rddInfoList.find { _.id == rddId }.map { rddInfo =>
+      getRDDStorageInfo(rddId, rddInfo, storageStatusList, includeDetails)
+    }
+  }
+
+  def getRDDStorageInfo(
+      rddId: Int,
+      rddInfo: RDDInfo,
+      storageStatusList: Seq[StorageStatus],
+      includeDetails: Boolean): RDDStorageInfo = {
+    val workers = storageStatusList.map { (rddId, _) }
+    val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList)
+    val blocks = storageStatusList
+      .flatMap { _.rddBlocksById(rddId) }
+      .sortWith { _._1.name < _._1.name }
+      .map { case (blockId, status) =>
+        (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown")))
+      }
+
+    val dataDistribution = if (includeDetails) {
+      Some(storageStatusList.map { status =>
+        new RDDDataDistribution(
+          address = status.blockManagerId.hostPort,
+          memoryUsed = status.memUsedByRdd(rddId),
+          memoryRemaining = status.memRemaining,
+          diskUsed = status.diskUsedByRdd(rddId)
+        ) } )
+    } else {
+      None
+    }
+    val partitions = if (includeDetails) {
+      Some(blocks.map { case (id, block, locations) =>
+        new RDDPartitionInfo(
+          blockName = id.name,
+          storageLevel = block.storageLevel.description,
+          memoryUsed = block.memSize,
+          diskUsed = block.diskSize,
+          executors = locations
+        )
+      } )
+    } else {
+      None
+    }
+
+    new RDDStorageInfo(
+      id = rddId,
+      name = rddInfo.name,
+      numPartitions = rddInfo.numPartitions,
+      numCachedPartitions = rddInfo.numCachedPartitions,
+      storageLevel = rddInfo.storageLevel.description,
+      memoryUsed = rddInfo.memSize,
+      diskUsed = rddInfo.diskSize,
+      dataDistribution = dataDistribution,
+      partitions = partitions
+    )
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
new file mode 100644
index 0000000..5060858
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.{Arrays, Date, List => JList}
+import javax.ws.rs.{GET, PathParam, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.executor.{InputMetrics => InternalInputMetrics, OutputMetrics => InternalOutputMetrics, ShuffleReadMetrics => InternalShuffleReadMetrics, ShuffleWriteMetrics => InternalShuffleWriteMetrics, TaskMetrics => InternalTaskMetrics}
+import org.apache.spark.scheduler.{AccumulableInfo => InternalAccumulableInfo, StageInfo}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData}
+import org.apache.spark.util.Distribution
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class AllStagesResource(ui: SparkUI) {
+
+  @GET
+  def stageList(@QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] = {
+    val listener = ui.jobProgressListener
+    val stageAndStatus = AllStagesResource.stagesAndStatus(ui)
+    val adjStatuses = {
+      if (statuses.isEmpty()) {
+        Arrays.asList(StageStatus.values(): _*)
+      } else {
+        statuses
+      }
+    }
+    for {
+      (status, stageList) <- stageAndStatus
+      stageInfo: StageInfo <- stageList if adjStatuses.contains(status)
+      stageUiData: StageUIData <- listener.synchronized {
+        listener.stageIdToData.get((stageInfo.stageId, stageInfo.attemptId))
+      }
+    } yield {
+      AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false)
+    }
+  }
+}
+
+private[v1] object AllStagesResource {
+  def stageUiToStageData(
+      status: StageStatus,
+      stageInfo: StageInfo,
+      stageUiData: StageUIData,
+      includeDetails: Boolean): StageData = {
+
+    val taskData = if (includeDetails) {
+      Some(stageUiData.taskData.map { case (k, v) => k -> convertTaskData(v) } )
+    } else {
+      None
+    }
+    val executorSummary = if (includeDetails) {
+      Some(stageUiData.executorSummary.map { case (k, summary) =>
+        k -> new ExecutorStageSummary(
+          taskTime = summary.taskTime,
+          failedTasks = summary.failedTasks,
+          succeededTasks = summary.succeededTasks,
+          inputBytes = summary.inputBytes,
+          outputBytes = summary.outputBytes,
+          shuffleRead = summary.shuffleRead,
+          shuffleWrite = summary.shuffleWrite,
+          memoryBytesSpilled = summary.memoryBytesSpilled,
+          diskBytesSpilled = summary.diskBytesSpilled
+        )
+      })
+    } else {
+      None
+    }
+
+    val accumulableInfo = stageUiData.accumulables.values.map { convertAccumulableInfo }.toSeq
+
+    new StageData(
+      status = status,
+      stageId = stageInfo.stageId,
+      attemptId = stageInfo.attemptId,
+      numActiveTasks = stageUiData.numActiveTasks,
+      numCompleteTasks = stageUiData.numCompleteTasks,
+      numFailedTasks = stageUiData.numFailedTasks,
+      executorRunTime = stageUiData.executorRunTime,
+      inputBytes = stageUiData.inputBytes,
+      inputRecords = stageUiData.inputRecords,
+      outputBytes = stageUiData.outputBytes,
+      outputRecords = stageUiData.outputRecords,
+      shuffleReadBytes = stageUiData.shuffleReadTotalBytes,
+      shuffleReadRecords = stageUiData.shuffleReadRecords,
+      shuffleWriteBytes = stageUiData.shuffleWriteBytes,
+      shuffleWriteRecords = stageUiData.shuffleWriteRecords,
+      memoryBytesSpilled = stageUiData.memoryBytesSpilled,
+      diskBytesSpilled = stageUiData.diskBytesSpilled,
+      schedulingPool = stageUiData.schedulingPool,
+      name = stageInfo.name,
+      details = stageInfo.details,
+      accumulatorUpdates = accumulableInfo,
+      tasks = taskData,
+      executorSummary = executorSummary
+    )
+  }
+
+  def stagesAndStatus(ui: SparkUI): Seq[(StageStatus, Seq[StageInfo])] = {
+    val listener = ui.jobProgressListener
+    listener.synchronized {
+      Seq(
+        StageStatus.ACTIVE -> listener.activeStages.values.toSeq,
+        StageStatus.COMPLETE -> listener.completedStages.reverse.toSeq,
+        StageStatus.FAILED -> listener.failedStages.reverse.toSeq,
+        StageStatus.PENDING -> listener.pendingStages.values.toSeq
+      )
+    }
+  }
+
+  def convertTaskData(uiData: TaskUIData): TaskData = {
+    new TaskData(
+      taskId = uiData.taskInfo.taskId,
+      index = uiData.taskInfo.index,
+      attempt = uiData.taskInfo.attempt,
+      launchTime = new Date(uiData.taskInfo.launchTime),
+      executorId = uiData.taskInfo.executorId,
+      host = uiData.taskInfo.host,
+      taskLocality = uiData.taskInfo.taskLocality.toString(),
+      speculative = uiData.taskInfo.speculative,
+      accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo },
+      errorMessage = uiData.errorMessage,
+      taskMetrics = uiData.taskMetrics.map { convertUiTaskMetrics }
+    )
+  }
+
+  def taskMetricDistributions(
+      allTaskData: Iterable[TaskUIData],
+      quantiles: Array[Double]): TaskMetricDistributions = {
+
+    val rawMetrics = allTaskData.flatMap{_.taskMetrics}.toSeq
+
+    def metricQuantiles(f: InternalTaskMetrics => Double): IndexedSeq[Double] =
+      Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles)
+
+    // We need to do a lot of similar munging to nested metrics here.  For each one,
+    // we want (a) extract the values for nested metrics (b) make a distribution for each metric
+    // (c) shove the distribution into the right field in our return type and (d) only return
+    // a result if the option is defined for any of the tasks.  MetricHelper is a little util
+    // to make it a little easier to deal w/ all of the nested options.  Mostly it lets us just
+    // implement one "build" method, which just builds the quantiles for each field.
+
+    val inputMetrics: Option[InputMetricDistributions] =
+      new MetricHelper[InternalInputMetrics, InputMetricDistributions](rawMetrics, quantiles) {
+        def getSubmetrics(raw: InternalTaskMetrics): Option[InternalInputMetrics] = {
+          raw.inputMetrics
+        }
+
+        def build: InputMetricDistributions = new InputMetricDistributions(
+          bytesRead = submetricQuantiles(_.bytesRead),
+          recordsRead = submetricQuantiles(_.recordsRead)
+        )
+      }.metricOption
+
+    val outputMetrics: Option[OutputMetricDistributions] =
+      new MetricHelper[InternalOutputMetrics, OutputMetricDistributions](rawMetrics, quantiles) {
+        def getSubmetrics(raw:InternalTaskMetrics): Option[InternalOutputMetrics] = {
+          raw.outputMetrics
+        }
+        def build: OutputMetricDistributions = new OutputMetricDistributions(
+          bytesWritten = submetricQuantiles(_.bytesWritten),
+          recordsWritten = submetricQuantiles(_.recordsWritten)
+        )
+      }.metricOption
+
+    val shuffleReadMetrics: Option[ShuffleReadMetricDistributions] =
+      new MetricHelper[InternalShuffleReadMetrics, ShuffleReadMetricDistributions](rawMetrics,
+        quantiles) {
+        def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleReadMetrics] = {
+          raw.shuffleReadMetrics
+        }
+        def build: ShuffleReadMetricDistributions = new ShuffleReadMetricDistributions(
+          readBytes = submetricQuantiles(_.totalBytesRead),
+          readRecords = submetricQuantiles(_.recordsRead),
+          remoteBytesRead = submetricQuantiles(_.remoteBytesRead),
+          remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched),
+          localBlocksFetched = submetricQuantiles(_.localBlocksFetched),
+          totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched),
+          fetchWaitTime = submetricQuantiles(_.fetchWaitTime)
+        )
+      }.metricOption
+
+    val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions] =
+      new MetricHelper[InternalShuffleWriteMetrics, ShuffleWriteMetricDistributions](rawMetrics,
+        quantiles) {
+        def getSubmetrics(raw: InternalTaskMetrics): Option[InternalShuffleWriteMetrics] = {
+          raw.shuffleWriteMetrics
+        }
+        def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions(
+          writeBytes = submetricQuantiles(_.shuffleBytesWritten),
+          writeRecords = submetricQuantiles(_.shuffleRecordsWritten),
+          writeTime = submetricQuantiles(_.shuffleWriteTime)
+        )
+      }.metricOption
+
+    new TaskMetricDistributions(
+      quantiles = quantiles,
+      executorDeserializeTime = metricQuantiles(_.executorDeserializeTime),
+      executorRunTime = metricQuantiles(_.executorRunTime),
+      resultSize = metricQuantiles(_.resultSize),
+      jvmGcTime = metricQuantiles(_.jvmGCTime),
+      resultSerializationTime = metricQuantiles(_.resultSerializationTime),
+      memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled),
+      diskBytesSpilled = metricQuantiles(_.diskBytesSpilled),
+      inputMetrics = inputMetrics,
+      outputMetrics = outputMetrics,
+      shuffleReadMetrics = shuffleReadMetrics,
+      shuffleWriteMetrics = shuffleWriteMetrics
+    )
+  }
+
+  def convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = {
+    new AccumulableInfo(acc.id, acc.name, acc.update, acc.value)
+  }
+
+  def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = {
+    new TaskMetrics(
+      executorDeserializeTime = internal.executorDeserializeTime,
+      executorRunTime = internal.executorRunTime,
+      resultSize = internal.resultSize,
+      jvmGcTime = internal.jvmGCTime,
+      resultSerializationTime = internal.resultSerializationTime,
+      memoryBytesSpilled = internal.memoryBytesSpilled,
+      diskBytesSpilled = internal.diskBytesSpilled,
+      inputMetrics = internal.inputMetrics.map { convertInputMetrics },
+      outputMetrics = Option(internal.outputMetrics).flatten.map { convertOutputMetrics },
+      shuffleReadMetrics = internal.shuffleReadMetrics.map { convertShuffleReadMetrics },
+      shuffleWriteMetrics = internal.shuffleWriteMetrics.map { convertShuffleWriteMetrics }
+    )
+  }
+
+  def convertInputMetrics(internal: InternalInputMetrics): InputMetrics = {
+    new InputMetrics(
+      bytesRead = internal.bytesRead,
+      recordsRead = internal.recordsRead
+    )
+  }
+
+  def convertOutputMetrics(internal: InternalOutputMetrics): OutputMetrics = {
+    new OutputMetrics(
+      bytesWritten = internal.bytesWritten,
+      recordsWritten = internal.recordsWritten
+    )
+  }
+
+  def convertShuffleReadMetrics(internal: InternalShuffleReadMetrics): ShuffleReadMetrics = {
+    new ShuffleReadMetrics(
+      remoteBlocksFetched = internal.remoteBlocksFetched,
+      localBlocksFetched = internal.localBlocksFetched,
+      fetchWaitTime = internal.fetchWaitTime,
+      remoteBytesRead = internal.remoteBytesRead,
+      totalBlocksFetched = internal.totalBlocksFetched,
+      recordsRead = internal.recordsRead
+    )
+  }
+
+  def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = {
+    new ShuffleWriteMetrics(
+      bytesWritten = internal.shuffleBytesWritten,
+      writeTime = internal.shuffleWriteTime,
+      recordsWritten = internal.shuffleRecordsWritten
+    )
+  }
+}
+
+/**
+ * Helper for getting distributions from nested metric types.  Many of the metrics we want are
+ * contained in options inside TaskMetrics (eg., ShuffleWriteMetrics). This makes it easy to handle
+ * the options (returning None if the metrics are all empty), and extract the quantiles for each
+ * metric.  After creating an instance, call metricOption to get the result type.
+ */
+private[v1] abstract class MetricHelper[I,O](
+    rawMetrics: Seq[InternalTaskMetrics],
+    quantiles: Array[Double]) {
+
+  def getSubmetrics(raw: InternalTaskMetrics): Option[I]
+
+  def build: O
+
+  val data: Seq[I] = rawMetrics.flatMap(getSubmetrics)
+
+  /** applies the given function to all input metrics, and returns the quantiles */
+  def submetricQuantiles(f: I => Double): IndexedSeq[Double] = {
+    Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles)
+  }
+
+  def metricOption: Option[O] = {
+    if (data.isEmpty) {
+      None
+    } else {
+      Some(build)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala
new file mode 100644
index 0000000..17b521f
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.{Arrays, Date, List => JList}
+import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.deploy.history.ApplicationHistoryInfo
+import org.apache.spark.deploy.master.{ApplicationInfo => InternalApplicationInfo}
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class ApplicationListResource(uiRoot: UIRoot) {
+
+  @GET
+  def appList(
+      @QueryParam("status") status: JList[ApplicationStatus],
+      @DefaultValue("2010-01-01") @QueryParam("minDate") minDate: SimpleDateParam,
+      @DefaultValue("3000-01-01") @QueryParam("maxDate") maxDate: SimpleDateParam)
+  : Iterator[ApplicationInfo] = {
+    val allApps = uiRoot.getApplicationInfoList
+    val adjStatus = {
+      if (status.isEmpty) {
+        Arrays.asList(ApplicationStatus.values(): _*)
+      } else {
+        status
+      }
+    }
+    val includeCompleted = adjStatus.contains(ApplicationStatus.COMPLETED)
+    val includeRunning = adjStatus.contains(ApplicationStatus.RUNNING)
+    allApps.filter { app =>
+      val anyRunning = app.attempts.exists(!_.completed)
+      // if any attempt is still running, we consider the app to also still be running
+      val statusOk = (!anyRunning && includeCompleted) ||
+        (anyRunning && includeRunning)
+      // keep the app if *any* attempts fall in the right time window
+      val dateOk = app.attempts.exists { attempt =>
+        attempt.startTime.getTime >= minDate.timestamp &&
+          attempt.startTime.getTime <= maxDate.timestamp
+      }
+      statusOk && dateOk
+    }
+  }
+}
+
+private[spark] object ApplicationsListResource {
+  def appHistoryInfoToPublicAppInfo(app: ApplicationHistoryInfo): ApplicationInfo = {
+    new ApplicationInfo(
+      id = app.id,
+      name = app.name,
+      attempts = app.attempts.map { internalAttemptInfo =>
+        new ApplicationAttemptInfo(
+          attemptId = internalAttemptInfo.attemptId,
+          startTime = new Date(internalAttemptInfo.startTime),
+          endTime = new Date(internalAttemptInfo.endTime),
+          sparkUser = internalAttemptInfo.sparkUser,
+          completed = internalAttemptInfo.completed
+        )
+      }
+    )
+  }
+
+  def convertApplicationInfo(
+      internal: InternalApplicationInfo,
+      completed: Boolean): ApplicationInfo = {
+    // standalone application info always has just one attempt
+    new ApplicationInfo(
+      id = internal.id,
+      name = internal.desc.name,
+      attempts = Seq(new ApplicationAttemptInfo(
+        attemptId = None,
+        startTime = new Date(internal.startTime),
+        endTime = new Date(internal.endTime),
+        sparkUser = internal.desc.user,
+        completed = completed
+      ))
+    )
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
new file mode 100644
index 0000000..8ad4656
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala
@@ -0,0 +1,36 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.{GET, PathParam, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.exec.ExecutorsPage
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class ExecutorListResource(ui: SparkUI) {
+
+  @GET
+  def executorList(): Seq[ExecutorSummary] = {
+    val listener = ui.executorsListener
+    val storageStatusList = listener.storageStatusList
+    (0 until storageStatusList.size).map { statusId =>
+      ExecutorsPage.getExecInfo(listener, statusId)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala
new file mode 100644
index 0000000..202a519
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.io.OutputStream
+import java.lang.annotation.Annotation
+import java.lang.reflect.Type
+import java.text.SimpleDateFormat
+import java.util.{Calendar, SimpleTimeZone}
+import javax.ws.rs.Produces
+import javax.ws.rs.core.{MediaType, MultivaluedMap}
+import javax.ws.rs.ext.{MessageBodyWriter, Provider}
+
+import com.fasterxml.jackson.annotation.JsonInclude
+import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature}
+
+/**
+ * This class converts the POJO metric responses into json, using jackson.
+ *
+ * This doesn't follow the standard jersey-jackson plugin options, because we want to stick
+ * with an old version of jersey (since we have it from yarn anyway) and don't want to pull in lots
+ * of dependencies from a new plugin.
+ *
+ * Note that jersey automatically discovers this class based on its package and its annotations.
+ */
+@Provider
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{
+
+  val mapper = new ObjectMapper() {
+    override def writeValueAsString(t: Any): String = {
+      super.writeValueAsString(t)
+    }
+  }
+  mapper.registerModule(com.fasterxml.jackson.module.scala.DefaultScalaModule)
+  mapper.enable(SerializationFeature.INDENT_OUTPUT)
+  mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL)
+  mapper.setDateFormat(JacksonMessageWriter.makeISODateFormat)
+
+  override def isWriteable(
+      aClass: Class[_],
+      `type`: Type,
+      annotations: Array[Annotation],
+      mediaType: MediaType): Boolean = {
+      true
+  }
+
+  override def writeTo(
+      t: Object,
+      aClass: Class[_],
+      `type`: Type,
+      annotations: Array[Annotation],
+      mediaType: MediaType,
+      multivaluedMap: MultivaluedMap[String, AnyRef],
+      outputStream: OutputStream): Unit = {
+    t match {
+      case ErrorWrapper(err) => outputStream.write(err.getBytes("utf-8"))
+      case _ => mapper.writeValue(outputStream, t)
+    }
+  }
+
+  override def getSize(
+      t: Object,
+      aClass: Class[_],
+      `type`: Type,
+      annotations: Array[Annotation],
+      mediaType: MediaType): Long = {
+    -1L
+  }
+}
+
+private[spark] object JacksonMessageWriter {
+  def makeISODateFormat: SimpleDateFormat = {
+    val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'")
+    val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT"))
+    iso8601.setCalendar(cal)
+    iso8601
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala
new file mode 100644
index 0000000..c3ec45f
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/JsonRootResource.scala
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.servlet.ServletContext
+import javax.ws.rs._
+import javax.ws.rs.core.{Context, Response}
+
+import com.sun.jersey.api.core.ResourceConfig
+import com.sun.jersey.spi.container.servlet.ServletContainer
+import org.eclipse.jetty.server.handler.ContextHandler
+import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
+
+import org.apache.spark.SecurityManager
+import org.apache.spark.ui.SparkUI
+
+/**
+ * Main entry point for serving spark application metrics as json, using JAX-RS.
+ *
+ * Each resource should have endpoints that return **public** classes defined in api.scala.  Mima
+ * binary compatibility checks ensure that we don't inadvertently make changes that break the api.
+ * The returned objects are automatically converted to json by jackson with JacksonMessageWriter.
+ * In addition, there are a number of tests in HistoryServerSuite that compare the json to "golden
+ * files".  Any changes and additions should be reflected there as well -- see the notes in
+ * HistoryServerSuite.
+ */
+@Path("/v1")
+private[v1] class JsonRootResource extends UIRootFromServletContext {
+
+  @Path("applications")
+  def getApplicationList(): ApplicationListResource = {
+    new ApplicationListResource(uiRoot)
+  }
+
+  @Path("applications/{appId}")
+  def getApplication(): OneApplicationResource = {
+    new OneApplicationResource(uiRoot)
+  }
+
+  @Path("applications/{appId}/{attemptId}/jobs")
+  def getJobs(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): AllJobsResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new AllJobsResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/jobs")
+  def getJobs(@PathParam("appId") appId: String): AllJobsResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new AllJobsResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/jobs/{jobId: \\d+}")
+  def getJob(@PathParam("appId") appId: String): OneJobResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new OneJobResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/jobs/{jobId: \\d+}")
+  def getJob(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): OneJobResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new OneJobResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/executors")
+  def getExecutors(@PathParam("appId") appId: String): ExecutorListResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new ExecutorListResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/executors")
+  def getExecutors(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): ExecutorListResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new ExecutorListResource(ui)
+    }
+  }
+
+
+  @Path("applications/{appId}/stages")
+  def getStages(@PathParam("appId") appId: String): AllStagesResource= {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new AllStagesResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/stages")
+  def getStages(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): AllStagesResource= {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new AllStagesResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/stages/{stageId: \\d+}")
+  def getStage(@PathParam("appId") appId: String): OneStageResource= {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new OneStageResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/stages/{stageId: \\d+}")
+  def getStage(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): OneStageResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new OneStageResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/storage/rdd")
+  def getRdds(@PathParam("appId") appId: String): AllRDDResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new AllRDDResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/storage/rdd")
+  def getRdds(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): AllRDDResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new AllRDDResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/storage/rdd/{rddId: \\d+}")
+  def getRdd(@PathParam("appId") appId: String): OneRDDResource = {
+    uiRoot.withSparkUI(appId, None) { ui =>
+      new OneRDDResource(ui)
+    }
+  }
+
+  @Path("applications/{appId}/{attemptId}/storage/rdd/{rddId: \\d+}")
+  def getRdd(
+      @PathParam("appId") appId: String,
+      @PathParam("attemptId") attemptId: String): OneRDDResource = {
+    uiRoot.withSparkUI(appId, Some(attemptId)) { ui =>
+      new OneRDDResource(ui)
+    }
+  }
+
+}
+
+private[spark] object JsonRootResource {
+
+  def getJsonServlet(uiRoot: UIRoot): ServletContextHandler = {
+    val jerseyContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS)
+    jerseyContext.setContextPath("/json")
+    val holder:ServletHolder = new ServletHolder(classOf[ServletContainer])
+    holder.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
+      "com.sun.jersey.api.core.PackagesResourceConfig")
+    holder.setInitParameter("com.sun.jersey.config.property.packages",
+      "org.apache.spark.status.api.v1")
+    holder.setInitParameter(ResourceConfig.PROPERTY_CONTAINER_REQUEST_FILTERS,
+      classOf[SecurityFilter].getCanonicalName)
+    UIRootFromServletContext.setUiRoot(jerseyContext, uiRoot)
+    jerseyContext.addServlet(holder, "/*")
+    jerseyContext
+  }
+}
+
+/**
+ * This trait is shared by the all the root containers for application UI information --
+ * the HistoryServer, the Master UI, and the application UI.  This provides the common
+ * interface needed for them all to expose application info as json.
+ */
+private[spark] trait UIRoot {
+  def getSparkUI(appKey: String): Option[SparkUI]
+  def getApplicationInfoList: Iterator[ApplicationInfo]
+
+  /**
+   * Get the spark UI with the given appID, and apply a function
+   * to it.  If there is no such app, throw an appropriate exception
+   */
+  def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = {
+    val appKey = attemptId.map(appId + "/" + _).getOrElse(appId)
+    getSparkUI(appKey) match {
+      case Some(ui) =>
+        f(ui)
+      case None => throw new NotFoundException("no such app: " + appId)
+    }
+  }
+  def securityManager: SecurityManager
+}
+
+private[v1] object UIRootFromServletContext {
+
+  private val attribute = getClass.getCanonicalName
+
+  def setUiRoot(contextHandler: ContextHandler, uiRoot: UIRoot): Unit = {
+    contextHandler.setAttribute(attribute, uiRoot)
+  }
+
+  def getUiRoot(context: ServletContext): UIRoot = {
+    context.getAttribute(attribute).asInstanceOf[UIRoot]
+  }
+}
+
+private[v1] trait UIRootFromServletContext {
+  @Context
+  var servletContext: ServletContext = _
+
+  def uiRoot: UIRoot = UIRootFromServletContext.getUiRoot(servletContext)
+}
+
+private[v1] class NotFoundException(msg: String) extends WebApplicationException(
+  new NoSuchElementException(msg),
+    Response
+      .status(Response.Status.NOT_FOUND)
+      .entity(ErrorWrapper(msg))
+      .build()
+)
+
+private[v1] class BadParameterException(msg: String) extends WebApplicationException(
+  new IllegalArgumentException(msg),
+  Response
+    .status(Response.Status.BAD_REQUEST)
+    .entity(ErrorWrapper(msg))
+    .build()
+) {
+  def this(param: String, exp: String, actual: String) = {
+    this(raw"""Bad value for parameter "$param".  Expected a $exp, got "$actual"""")
+  }
+}
+
+/**
+ * Signal to JacksonMessageWriter to not convert the message into json (which would result in an
+ * extra set of quotes).
+ */
+private[v1] case class ErrorWrapper(s: String)

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
new file mode 100644
index 0000000..b5ef726
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.core.MediaType
+import javax.ws.rs.{Produces, PathParam, GET}
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneApplicationResource(uiRoot: UIRoot) {
+
+  @GET
+  def getApp(@PathParam("appId") appId: String): ApplicationInfo = {
+    val apps = uiRoot.getApplicationInfoList.find { _.id == appId }
+    apps.getOrElse(throw new NotFoundException("unknown app: " + appId))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala
new file mode 100644
index 0000000..6d8a60d
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.{PathParam, GET, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.UIData.JobUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneJobResource(ui: SparkUI) {
+
+  @GET
+  def oneJob(@PathParam("jobId") jobId: Int): JobData = {
+    val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] =
+      AllJobsResource.getStatusToJobs(ui)
+    val jobOpt = statusToJobs.map {_._2} .flatten.find { jobInfo => jobInfo.jobId == jobId}
+    jobOpt.map { job =>
+      AllJobsResource.convertJobData(job, ui.jobProgressListener, false)
+    }.getOrElse {
+      throw new NotFoundException("unknown job: " + jobId)
+    }
+  }
+
+}


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


[02/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1427397477963
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1427397477963 b/core/src/test/resources/spark-events/local-1427397477963
new file mode 100755
index 0000000..3bd251d
--- /dev/null
+++ b/core/src/test/resources/spark-events/local-1427397477963
@@ -0,0 +1,12083 @@
+{"Event":"SparkListenerLogStart","Spark Version":"1.4.0-SNAPSHOT"}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"<driver>","Host":"localhost","Port":53827},"Maximum Memory":257918238,"Timestamp":1427397478113}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.101","spark.eventLog.enabled":"true","spark.driver.port":"53825","spark.repl.class.uri":"http://192.168.1.101:53823","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.cleaner.ttl.MAP_OUTPUT_TRACKER":"1","spark.executor.id":"<driver>","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.101:53826","spark.tachyonStore.folderName":"spark-ba9af2c0-12a3-4d07-8f0a-2aded3ba3ded","spark.app.id":"local-1427397477963"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/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/irashid","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_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/github/spark","java.library.path":"/Users/irashid/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.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.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":"/","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_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.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.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.eventLog.enabled=true --conf spark.cleaner.ttl.MAP_OUTPUT_TRACKER=1 --class org.apache.spark.repl.Ma
 in spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/github/spark/tools/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-continuation-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/yarn/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-http-8.1.14.v20131031.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/github/spark/launcher/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/streaming/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/sql/hive/target/scala-2.10/classes/":
 "System Classpath","/Users/irashid/github/spark/core/target/jars/guava-14.0.1.jar":"System Classpath","/Users/irashid/github/spark/sql/core/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/repl/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/conf/":"System Classpath","/Users/irashid/github/spark/sql/hive-thriftserver/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/bagel/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-security-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-server-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/sql/catalyst/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-util-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-servlet-8.1.14.v20131031.jar":"System Classpath","/Us
 ers/irashid/github/spark/core/target/jars/jetty-io-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/mllib/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/assembly/target/scala-2.10/spark-assembly-1.4.0-SNAPSHOT-hadoop2.5.0.jar":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-plus-8.1.14.v20131031.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/github/spark/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/github/spark/graphx/target/scala-2.10/classes/":"System Classpath","/Users/irashid/github/spark/core/target/jars/jetty-util-6.1.26.jar":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1427397477963","Timestamp":1427397477184,"User":"irashid"}
+{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1427397514437,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at <console>:21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<
 console>:30)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:32)\n$line15.$read$$iwC$$iwC$$iwC.<init>(<console>:34)\n$line15.$read$$iwC$$iwC.<init>(<console>:36)\n$line15.$read$$iwC.<init>(<console>:38)\n$line15.$read.<init>(<console>:40)\n$line15.$read$.<init>(<console>:44)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.<init>(<console>:7)\n$line15.$eval$.<clinit>(<console>)\n$line15.$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:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"replStringOf at <console>:10","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":fa
 lse,"Deserialized":false,"Replication":1},"Number of Partitions":50,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:337)\n$line19.$eval$.<init>(<console>:10)\n$line19.$eval$.<clinit>(<console>)\n$line19.$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:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1338)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)\norg.apache.spark.repl.SparkILoop
 .reallyInterpret$1(SparkILoop.scala:856)\norg.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:901)\norg.apache.spark.repl.SparkILoop.command(SparkILoop.scala:813)\norg.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:656)\norg.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:664)\norg.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:669)\norg.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:996)","Accumulables":[]}],"Stage IDs":[0,1]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"map at <console>:21","Number of Tasks":1000,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":1000,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:287)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:21)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:26)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:28)\n$line15.$read$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:30)\n$line15.$read$$iwC$$iwC$$
 iwC$$iwC.<init>(<console>:32)\n$line15.$read$$iwC$$iwC$$iwC.<init>(<console>:34)\n$line15.$read$$iwC$$iwC.<init>(<console>:36)\n$line15.$read$$iwC.<init>(<console>:38)\n$line15.$read.<init>(<console>:40)\n$line15.$read$.<init>(<console>:44)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.<init>(<console>:7)\n$line15.$eval$.<clinit>(<console>)\n$line15.$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:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1427397514583,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1427397514599,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1427397514895,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1427397514897,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1427397514898,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514903,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":2709542,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514906,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1683487,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1427397514907,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1427397514602,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514908,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":273,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":438240,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1427397514908,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1427397514599,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514910,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":262,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2220286,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1427397514910,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":270,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1664784,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1427397514600,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514911,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":261,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":2178902,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1427397514583,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":16,"Executor Run Time":274,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":4718634,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1427397514601,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514916,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":15,"Executor Run Time":269,"Result Size":930,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1786420,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1427397514975,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1427397514910,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514975,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":430298,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1427397514979,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1427397514908,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514979,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":419405,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1427397514989,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514989,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":73,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":433438,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1427397514995,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1427397514898,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514996,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":91,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":479137,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1427397514997,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1427397514912,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397514998,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":79,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":478808,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1427397515002,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1427397514895,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":103,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":525454,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1427397515011,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1427397514907,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515012,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":101,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":593414,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1427397515012,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1427397514897,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515013,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":112,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":524832,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1427397515055,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1427397514975,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515056,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":78,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":504502,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1427397515059,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1427397515011,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":46,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":459758,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1427397515060,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1427397514979,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515060,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":80,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1349002,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1427397515062,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1427397514989,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515062,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":545262,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1427397515069,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1427397514995,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515070,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":72,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":561286,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1427397514997,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515074,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":74,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":492272,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1427397515012,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515075,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":511744,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1427397515082,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1427397515002,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515082,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":78,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":590793,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1427397515107,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515108,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":712959,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1427397515111,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1427397515060,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515112,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1133685,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1427397515113,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1427397515062,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515113,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":647955,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1427397515127,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1427397515059,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515127,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":65,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":487315,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1427397515128,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1427397515082,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515129,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":1173343,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1427397515134,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1427397515107,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515134,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":25,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":542166,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1427397515055,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":17403787,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1427397515069,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515135,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":63,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":870321,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1427397515145,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1427397515074,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515146,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":859166,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1427397515168,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515168,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":711567,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1427397515184,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1427397515113,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515184,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":506890,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1427397515188,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1427397515128,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515188,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":58,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1552336,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1427397515189,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1427397515134,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515189,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":52,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":502961,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1427397515198,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1427397515135,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515198,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":59,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":713958,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1427397515204,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1427397515168,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515205,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":33,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":775901,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1427397515207,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1427397515127,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515208,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":77,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":852701,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1427397515211,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1427397515111,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515211,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":98,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":4651370,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1427397515217,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1427397515145,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515218,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":69,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1798770,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1427397515242,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1427397515204,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515243,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":35,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9604,"Shuffle Write Time":411722,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1427397515251,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1427397515198,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515251,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":51,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9603,"Shuffle Write Time":888369,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1427397515252,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1427397515211,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515252,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":525960,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1427397515253,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1427397515184,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515253,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":66,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1901209,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1427397515255,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1427397515207,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515255,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":46,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":812239,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1427397515264,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1427397515189,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515264,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":52,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":373618,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1427397515265,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1427397515217,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515265,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":44,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1742885,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1427397515276,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1427397515242,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515276,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":31,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":486771,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1427397515281,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1427397515188,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515281,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":90,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1036134,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1427397515303,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1427397515252,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515303,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":49,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":733581,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1427397515304,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1427397515251,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515304,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":857448,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1427397515306,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1427397515255,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515306,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":50,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":822807,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1427397515308,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1427397515265,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515308,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":38,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":597860,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1427397515339,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1427397515276,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":61,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1682586,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1427397515343,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1427397515253,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515344,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":88,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9597,"Shuffle Write Time":1918631,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1427397515347,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1427397515306,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515347,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":39,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":570824,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1427397515362,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1427397515264,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515363,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":96,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":467801,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1427397515374,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1427397515281,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515374,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":91,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":730395,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1427397515375,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1427397515303,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515375,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":69,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9610,"Shuffle Write Time":885679,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1427397515394,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1427397515343,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515394,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":701822,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1427397515395,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1427397515362,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515395,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":30,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":724051,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1427397515405,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1427397515304,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515405,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":99,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9599,"Shuffle Write Time":1235948,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1427397515410,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1427397515308,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515410,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":100,"Result Size":930,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1249286,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1427397515413,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1427397515339,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515413,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":71,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9601,"Shuffle Write Time":1628301,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1427397515414,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1427397515375,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515415,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":37,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9598,"Shuffle Write Time":1319579,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1427397515435,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1427397515347,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515436,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":84,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":940133,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1427397515439,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1427397515395,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515440,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":43,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1161930,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1427397515445,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1427397515374,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515445,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":68,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9614,"Shuffle Write Time":4875074,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1427397515452,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1427397515405,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515453,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":45,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9600,"Shuffle Write Time":1422810,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1427397515454,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1427397515410,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1427397515454,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":42,"Result Size":930,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":9602,"Shuffle Write Time":1311510,"Shuffle Records Written":100}}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1427397515459,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1427397515413,"Executor ID":"<driver>","Host":"localhos

<TRUNCATED>

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


[06/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1
new file mode 100755
index 0000000..4794e56
--- /dev/null
+++ b/core/src/test/resources/spark-events/local-1422981759269/EVENT_LOG_1
@@ -0,0 +1,88 @@
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"<driver>","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1422981759407}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1422981759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/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.7","java.vm.specification.name":"
 Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","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.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/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":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Li
 brary/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chica
 go","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/
 Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981759269","Timestamp":1422981758277,"User":"irashid"}
+{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\n
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.ref
 lect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"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":1422981762082,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"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":1422981762083,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"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":1422981762085,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\n
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981762069,"Completion Time":1422981762637,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n
 $line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at <console>:17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Nu
 mber of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line11.$read$$iwC$$iwC.<init>(<console>:24)\n$line11.$read$$iwC.<init>(<console>:26)\n$line11.$read.<init>(<console>:28)\n$line11.$read$.<init>(<console>:32)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\
 norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10
 .$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10
 .$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981763578,"Completion Time":1422981764005,"Accumulables":[]}}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at <console>:17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line11.$read$$iwC$$iwC.<init>(<console>:24)\n$line11.$read$$iwC.<init>(<console>:26)\n$line11.$read.<init>(<console>:28)\n$line11.$read$.<init>(<console>:32)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java
 :57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at <console>:17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line11.$read$$iwC$$iwC.<init>(<console>:24)\n$line11.$read$$iwC.<init>(<console>:26)\n$line11.$read.<init>(<console>:28)\n$line11.$read$.<init>(<console>:32)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java
 :57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764014,"Completion Time":1422981764045,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n
 $line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at <console>:19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replicatio
 n":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line13.$read$$iwC$$iwC$$iwC.<init>(<console>:24)\n$line13.$read$$iwC$$iwC.<init>(<console>:26)\n$line13.$read$$iwC.<init>(<console>:28)\n$line13.$read.<init>(<console>:30)\n$line13.$read$.<init>(<console>:34)\n$line13.$read$.<clinit>(<console>)\n$line13.$eval$.<init>(<console>:7)\n$line13.$eval$.<clinit>(<console>)\n$line13.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAc
 cessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at <console>:19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line13.$read$$iwC$$iwC$$iwC.<init>(<console>:24)\n$line13.$read$$iwC$$iwC.<init>(<console>:26)\n$line13.$read$$iwC.<init>(<console>:28)\n$line13.$read.<init>(<console>:30)\n$line13.$read$.<init>(<console>:34)\n$line13.$read$.<cl
 init>(<console>)\n$line13.$eval$.<init>(<console>:7)\n$line13.$eval$.<clinit>(<console>)\n$line13.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at <console>:19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line13.$read$$iwC$$iwC$$iwC.<init>(<console>:24)\n$line13.$read$$iwC$$iwC.<init>(<console>:26)\n$line13.$read$$iwC.<init>(<console>:28)\n$line13.$read.<init>(<console>:30)\n$line13.$read$.<init>(<console>:34)\n$line13.$read$.<cl
 init>(<console>)\n$line13.$eval$.<init>(<console>:7)\n$line13.$eval$.<clinit>(<console>)\n$line13.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764396,"Completion Time":1422981764648,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line15.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line15.$read$$iwC$$iwC.<init>(<console>:24)\n$line15.$read$$iwC.<init>(<console>:26)\n$line15.$read.<init
 >(<console>:28)\n$line15.$read$.<init>(<console>:32)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.<init>(<console>:7)\n$line15.$eval$.<clinit>(<console>)\n$line15.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line15.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line15.$read$$iwC$$iwC.<init>(<console>:24)\n$line15.$read$$iwC.<init>(<console>:26)\n$line15.$read.<init>(<cons
 ole>:28)\n$line15.$read$.<init>(<console>:32)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.<init>(<console>:7)\n$line15.$eval$.<clinit>(<console>)\n$line15.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line15.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line15.$read$$iwC$$iwC.<init>(<console>:24)\n$line15.$read$$iwC.<init>(<console>:26)\n$line15.$read.<init>(<cons
 ole>:28)\n$line15.$read$.<init>(<console>:32)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.<init>(<console>:7)\n$line15.$eval$.<clinit>(<console>)\n$line15.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981765026,"Completion Time":1422981765050,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1422981759269/SPARK_VERSION_1.2.0
new file mode 100755
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1422981780767/APPLICATION_COMPLETE
new file mode 100755
index 0000000..e69de29


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


[08/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation
new file mode 100644
index 0000000..31ac9be
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=complete/json_expectation
@@ -0,0 +1,67 @@
+[ {
+  "status" : "COMPLETE",
+  "stageId" : 3,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 162,
+  "inputBytes" : 160,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "count at <console>:17",
+  "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line19.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line19.$read$$iwC$$iwC.<init>(<console>:24)\n$line19.$read$$iwC.<init>(<console>:26)\n$line19.$read.<init>(<console>:28)\n$line19.$read$.<init>(<console>:32)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.<init>(<console>:7)\n$line19.$eval$.<clinit>(<console>)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.Spark
 IMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ]
+}, {
+  "status" : "COMPLETE",
+  "stageId" : 1,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 3476,
+  "inputBytes" : 28000128,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 13180,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "map at <console>:14",
+  "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIM
 ain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ]
+}, {
+  "status" : "COMPLETE",
+  "stageId" : 0,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 4338,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "count at <console>:15",
+  "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.inte
 rpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ]
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation
new file mode 100644
index 0000000..bff6a4f
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages?status=failed/json_expectation
@@ -0,0 +1,23 @@
+[ {
+  "status" : "FAILED",
+  "stageId" : 2,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 7,
+  "numFailedTasks" : 1,
+  "executorRunTime" : 278,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "count at <console>:20",
+  "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.Spark
 IMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ]
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation
new file mode 100644
index 0000000..38b5328
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/0/json_expectation
@@ -0,0 +1,64 @@
+{
+  "id" : 0,
+  "name" : "0",
+  "numPartitions" : 8,
+  "numCachedPartitions" : 8,
+  "storageLevel" : "Memory Deserialized 1x Replicated",
+  "memoryUsed" : 28000128,
+  "diskUsed" : 0,
+  "dataDistribution" : [ {
+    "address" : "localhost:57971",
+    "memoryUsed" : 28000128,
+    "memoryRemaining" : 250302428,
+    "diskUsed" : 0
+  } ],
+  "partitions" : [ {
+    "blockName" : "rdd_0_0",
+    "storageLevel" : "Memory Deserialized 1x Replicated",
+    "memoryUsed" : 3500016,
+    "diskUsed" : 0,
+    "executors" : [ "localhost:57971" ]
+  }, {
+    "blockName" : "rdd_0_1",
+    "storageLevel" : "Memory Deserialized 1x Replicated",
+    "memoryUsed" : 3500016,
+    "diskUsed" : 0,
+    "executors" : [ "localhost:57971" ]
+  }, {
+    "blockName" : "rdd_0_2",
+    "storageLevel" : "Memory Deserialized 1x Replicated",
+    "memoryUsed" : 3500016,
+    "diskUsed" : 0,
+    "executors" : [ "localhost:57971" ]
+  }, {
+    "blockName" : "rdd_0_3",
+    "storageLevel" : "Memory Deserialized 1x Replicated",
+    "memoryUsed" : 3500016,
+    "diskUsed" : 0,
+    "executors" : [ "localhost:57971" ]
+  }, {
+    "blockName" : "rdd_0_4",
+    "storageLevel" : "Memory Deserialized 1x Replicated",
+    "memoryUsed" : 3500016,
+    "diskUsed" : 0,
+    "executors" : [ "localhost:57971" ]
+  }, {
+    "blockName" : "rdd_0_5",
+    "storageLevel" : "Memory Deserialized 1x Replicated",
+    "memoryUsed" : 3500016,
+    "diskUsed" : 0,
+    "executors" : [ "localhost:57971" ]
+  }, {
+    "blockName" : "rdd_0_6",
+    "storageLevel" : "Memory Deserialized 1x Replicated",
+    "memoryUsed" : 3500016,
+    "diskUsed" : 0,
+    "executors" : [ "localhost:57971" ]
+  }, {
+    "blockName" : "rdd_0_7",
+    "storageLevel" : "Memory Deserialized 1x Replicated",
+    "memoryUsed" : 3500016,
+    "diskUsed" : 0,
+    "executors" : [ "localhost:57971" ]
+  } ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation
new file mode 100644
index 0000000..f79a310
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/storage/rdd/json_expectation
@@ -0,0 +1,9 @@
+[ {
+  "id" : 0,
+  "name" : "0",
+  "numPartitions" : 8,
+  "numCachedPartitions" : 8,
+  "storageLevel" : "Memory Deserialized 1x Replicated",
+  "memoryUsed" : 28000128,
+  "diskUsed" : 0
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation
new file mode 100644
index 0000000..2e92e1f
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/jobs/json_expectation
@@ -0,0 +1,15 @@
+[ {
+  "jobId" : 0,
+  "name" : "foreach at <console>:15",
+  "stageIds" : [ 0 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation
new file mode 100644
index 0000000..32d5731
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/json_expectation
@@ -0,0 +1,242 @@
+{
+  "status" : "COMPLETE",
+  "stageId" : 0,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 120,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "foreach at <console>:15",
+  "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$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:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.in
 terpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "value" : "5050"
+  } ],
+  "tasks" : {
+    "2" : {
+      "taskId" : 2,
+      "index" : 2,
+      "attempt" : 0,
+      "launchTime" : "2015-03-16T19:25:36.522GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ {
+        "id" : 1,
+        "name" : "my counter",
+        "update" : "378",
+        "value" : "378"
+      } ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 13,
+        "executorRunTime" : 15,
+        "resultSize" : 697,
+        "jvmGcTime" : 0,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0
+      }
+    },
+    "5" : {
+      "taskId" : 5,
+      "index" : 5,
+      "attempt" : 0,
+      "launchTime" : "2015-03-16T19:25:36.523GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ {
+        "id" : 1,
+        "name" : "my counter",
+        "update" : "897",
+        "value" : "3750"
+      } ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 12,
+        "executorRunTime" : 15,
+        "resultSize" : 697,
+        "jvmGcTime" : 0,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0
+      }
+    },
+    "4" : {
+      "taskId" : 4,
+      "index" : 4,
+      "attempt" : 0,
+      "launchTime" : "2015-03-16T19:25:36.522GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ {
+        "id" : 1,
+        "name" : "my counter",
+        "update" : "678",
+        "value" : "2853"
+      } ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 12,
+        "executorRunTime" : 15,
+        "resultSize" : 697,
+        "jvmGcTime" : 0,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0
+      }
+    },
+    "7" : {
+      "taskId" : 7,
+      "index" : 7,
+      "attempt" : 0,
+      "launchTime" : "2015-03-16T19:25:36.524GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ {
+        "id" : 1,
+        "name" : "my counter",
+        "update" : "1222",
+        "value" : "4972"
+      } ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 12,
+        "executorRunTime" : 15,
+        "resultSize" : 697,
+        "jvmGcTime" : 0,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0
+      }
+    },
+    "1" : {
+      "taskId" : 1,
+      "index" : 1,
+      "attempt" : 0,
+      "launchTime" : "2015-03-16T19:25:36.521GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ {
+        "id" : 1,
+        "name" : "my counter",
+        "update" : "247",
+        "value" : "2175"
+      } ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 14,
+        "executorRunTime" : 15,
+        "resultSize" : 697,
+        "jvmGcTime" : 0,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0
+      }
+    },
+    "3" : {
+      "taskId" : 3,
+      "index" : 3,
+      "attempt" : 0,
+      "launchTime" : "2015-03-16T19:25:36.522GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ {
+        "id" : 1,
+        "name" : "my counter",
+        "update" : "572",
+        "value" : "950"
+      } ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 13,
+        "executorRunTime" : 15,
+        "resultSize" : 697,
+        "jvmGcTime" : 0,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0
+      }
+    },
+    "6" : {
+      "taskId" : 6,
+      "index" : 6,
+      "attempt" : 0,
+      "launchTime" : "2015-03-16T19:25:36.523GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ {
+        "id" : 1,
+        "name" : "my counter",
+        "update" : "978",
+        "value" : "1928"
+      } ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 12,
+        "executorRunTime" : 15,
+        "resultSize" : 697,
+        "jvmGcTime" : 0,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0
+      }
+    },
+    "0" : {
+      "taskId" : 0,
+      "index" : 0,
+      "attempt" : 0,
+      "launchTime" : "2015-03-16T19:25:36.515GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ {
+        "id" : 1,
+        "name" : "my counter",
+        "update" : "78",
+        "value" : "5050"
+      } ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 14,
+        "executorRunTime" : 15,
+        "resultSize" : 697,
+        "jvmGcTime" : 0,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0
+      }
+    }
+  },
+  "executorSummary" : {
+    "<driver>" : {
+      "taskTime" : 418,
+      "failedTasks" : 0,
+      "succeededTasks" : 8,
+      "inputBytes" : 0,
+      "outputBytes" : 0,
+      "shuffleRead" : 0,
+      "shuffleWrite" : 0,
+      "memoryBytesSpilled" : 0,
+      "diskBytesSpilled" : 0
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation
new file mode 100644
index 0000000..c3febc5
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/0/0/taskList/json_expectation
@@ -0,0 +1,193 @@
+[ {
+  "taskId" : 0,
+  "index" : 0,
+  "attempt" : 0,
+  "launchTime" : "2015-03-16T19:25:36.515GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "78",
+    "value" : "5050"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 14,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 1,
+  "index" : 1,
+  "attempt" : 0,
+  "launchTime" : "2015-03-16T19:25:36.521GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "247",
+    "value" : "2175"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 14,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 2,
+  "index" : 2,
+  "attempt" : 0,
+  "launchTime" : "2015-03-16T19:25:36.522GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "378",
+    "value" : "378"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 13,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 3,
+  "index" : 3,
+  "attempt" : 0,
+  "launchTime" : "2015-03-16T19:25:36.522GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "572",
+    "value" : "950"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 13,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 4,
+  "index" : 4,
+  "attempt" : 0,
+  "launchTime" : "2015-03-16T19:25:36.522GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "678",
+    "value" : "2853"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 12,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 1,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 5,
+  "index" : 5,
+  "attempt" : 0,
+  "launchTime" : "2015-03-16T19:25:36.523GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "897",
+    "value" : "3750"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 12,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 6,
+  "index" : 6,
+  "attempt" : 0,
+  "launchTime" : "2015-03-16T19:25:36.523GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "978",
+    "value" : "1928"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 12,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 7,
+  "index" : 7,
+  "attempt" : 0,
+  "launchTime" : "2015-03-16T19:25:36.524GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "1222",
+    "value" : "4972"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 12,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation
new file mode 100644
index 0000000..79ccacd
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/1/stages/json_expectation
@@ -0,0 +1,27 @@
+[ {
+  "status" : "COMPLETE",
+  "stageId" : 0,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 120,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "foreach at <console>:15",
+  "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$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:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.in
 terpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "value" : "5050"
+  } ]
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation
new file mode 100644
index 0000000..2e92e1f
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/jobs/json_expectation
@@ -0,0 +1,15 @@
+[ {
+  "jobId" : 0,
+  "name" : "foreach at <console>:15",
+  "stageIds" : [ 0 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation
new file mode 100644
index 0000000..56d667d
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/2/stages/0/0/taskList/json_expectation
@@ -0,0 +1,193 @@
+[ {
+  "taskId" : 0,
+  "index" : 0,
+  "attempt" : 0,
+  "launchTime" : "2015-03-17T23:12:16.515GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "78",
+    "value" : "5050"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 14,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 1,
+  "index" : 1,
+  "attempt" : 0,
+  "launchTime" : "2015-03-17T23:12:16.521GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "247",
+    "value" : "2175"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 14,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 2,
+  "index" : 2,
+  "attempt" : 0,
+  "launchTime" : "2015-03-17T23:12:16.522GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "378",
+    "value" : "378"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 13,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 3,
+  "index" : 3,
+  "attempt" : 0,
+  "launchTime" : "2015-03-17T23:12:16.522GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "572",
+    "value" : "950"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 13,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 4,
+  "index" : 4,
+  "attempt" : 0,
+  "launchTime" : "2015-03-17T23:12:16.522GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "678",
+    "value" : "2853"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 12,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 1,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 5,
+  "index" : 5,
+  "attempt" : 0,
+  "launchTime" : "2015-03-17T23:12:16.523GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "897",
+    "value" : "3750"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 12,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 6,
+  "index" : 6,
+  "attempt" : 0,
+  "launchTime" : "2015-03-17T23:12:16.523GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "978",
+    "value" : "1928"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 12,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+}, {
+  "taskId" : 7,
+  "index" : 7,
+  "attempt" : 0,
+  "launchTime" : "2015-03-17T23:12:16.524GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ {
+    "id" : 1,
+    "name" : "my counter",
+    "update" : "1222",
+    "value" : "4972"
+  } ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 12,
+    "executorRunTime" : 15,
+    "resultSize" : 697,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 2,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0
+  }
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation
new file mode 100644
index 0000000..8f3d716
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1426533911241/json_expectation
@@ -0,0 +1,17 @@
+{
+  "id" : "local-1426533911241",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "attemptId" : "2",
+    "startTime" : "2015-03-17T23:11:50.242GMT",
+    "endTime" : "2015-03-17T23:12:25.177GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  }, {
+    "attemptId" : "1",
+    "startTime" : "2015-03-16T19:25:10.242GMT",
+    "endTime" : "2015-03-16T19:25:45.177GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation
new file mode 100644
index 0000000..140c76d
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList/json_expectation
@@ -0,0 +1,481 @@
+[ {
+  "taskId" : 5010,
+  "index" : 0,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.571GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 318464,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5011,
+  "index" : 1,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.571GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 271674,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5012,
+  "index" : 2,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.571GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 274899,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5013,
+  "index" : 3,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.571GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 11,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9597,
+      "writeTime" : 315224,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5014,
+  "index" : 4,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.571GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 12,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 316561,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5015,
+  "index" : 5,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.571GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 269443,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5016,
+  "index" : 6,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.571GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 275977,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5017,
+  "index" : 7,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.571GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 267412,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5018,
+  "index" : 8,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 310279,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5019,
+  "index" : 9,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 319028,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5020,
+  "index" : 10,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 306724,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5021,
+  "index" : 11,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 280729,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5022,
+  "index" : 12,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 6,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 273837,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5023,
+  "index" : 13,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 6,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 282108,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5024,
+  "index" : 14,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9604,
+      "writeTime" : 266308,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5025,
+  "index" : 15,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 8,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 330830,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5026,
+  "index" : 16,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 247775,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5027,
+  "index" : 17,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 289503,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5028,
+  "index" : 18,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.592GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 355545,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5029,
+  "index" : 19,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.593GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 339081,
+      "recordsWritten" : 100
+    }
+  }
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation
new file mode 100644
index 0000000..0fa1ac2
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?offset=10&length=50/json_expectation
@@ -0,0 +1,1201 @@
+[ {
+  "taskId" : 5020,
+  "index" : 10,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 306724,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5021,
+  "index" : 11,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 280729,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5022,
+  "index" : 12,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 6,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 273837,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5023,
+  "index" : 13,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 6,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 282108,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5024,
+  "index" : 14,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9604,
+      "writeTime" : 266308,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5025,
+  "index" : 15,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 8,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 330830,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5026,
+  "index" : 16,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 247775,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5027,
+  "index" : 17,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 289503,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5028,
+  "index" : 18,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.592GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 355545,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5029,
+  "index" : 19,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.593GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 339081,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5030,
+  "index" : 20,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.593GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 9,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 367295,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5031,
+  "index" : 21,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.593GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 9,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9597,
+      "writeTime" : 493701,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5032,
+  "index" : 22,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.593GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 345941,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5033,
+  "index" : 23,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.593GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 301382,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5034,
+  "index" : 24,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.594GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9603,
+      "writeTime" : 340547,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5035,
+  "index" : 25,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.600GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 324374,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5036,
+  "index" : 26,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.600GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9603,
+      "writeTime" : 321083,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5037,
+  "index" : 27,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.601GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 318662,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5038,
+  "index" : 28,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.601GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 334376,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5039,
+  "index" : 29,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.601GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 287042,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5040,
+  "index" : 30,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.601GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 332938,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5041,
+  "index" : 31,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.602GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 4,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9603,
+      "writeTime" : 255099,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5042,
+  "index" : 32,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.602GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 297534,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5043,
+  "index" : 33,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.607GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 265929,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5044,
+  "index" : 34,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.607GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 279851,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5045,
+  "index" : 35,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.608GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 278965,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5046,
+  "index" : 36,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.609GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 340518,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5047,
+  "index" : 37,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.609GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9597,
+      "writeTime" : 402877,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5048,
+  "index" : 38,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.609GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 265447,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5049,
+  "index" : 39,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.611GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 3,
+    "executorRunTime" : 23,
+    "resultSize" : 930,
+    "jvmGcTime" : 16,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 304745,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5050,
+  "index" : 40,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.615GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 24,
+    "resultSize" : 930,
+    "jvmGcTime" : 16,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 302371,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5051,
+  "index" : 41,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.615GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 25,
+    "resultSize" : 930,
+    "jvmGcTime" : 16,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 296177,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5052,
+  "index" : 42,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.615GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 24,
+    "resultSize" : 930,
+    "jvmGcTime" : 16,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 962033,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5053,
+  "index" : 43,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.616GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 26,
+    "resultSize" : 930,
+    "jvmGcTime" : 16,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 310335,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5054,
+  "index" : 44,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.616GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 110,
+    "resultSize" : 930,
+    "jvmGcTime" : 22,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 299315,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5055,
+  "index" : 45,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.617GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 25,
+    "resultSize" : 930,
+    "jvmGcTime" : 16,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9603,
+      "writeTime" : 307821,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5056,
+  "index" : 46,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.617GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 17,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9604,
+      "writeTime" : 306911,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5057,
+  "index" : 47,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.639GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 274191,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5058,
+  "index" : 48,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.640GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 48,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 10442141,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5059,
+  "index" : 49,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.641GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 300675,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5060,
+  "index" : 50,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.642GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 24,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 812196,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5061,
+  "index" : 51,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.642GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 6,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 313090,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5062,
+  "index" : 52,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.642GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 17,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 1572828,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5063,
+  "index" : 53,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.643GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9597,
+      "writeTime" : 312481,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5064,
+  "index" : 54,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.646GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 13,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 296976,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5065,
+  "index" : 55,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.650GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 670271,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5066,
+  "index" : 56,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.650GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 10,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 1240309,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5067,
+  "index" : 57,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.650GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 9,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 678632,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5068,
+  "index" : 58,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.659GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 302888,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5069,
+  "index" : 59,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.660GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 7,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9610,
+      "writeTime" : 383153,
+      "recordsWritten" : 100
+    }
+  }
+} ]
\ No newline at end of file


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


[07/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation
new file mode 100644
index 0000000..63b1316
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=-runtime/json_expectation
@@ -0,0 +1,481 @@
+[ {
+  "taskId" : 5186,
+  "index" : 176,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.880GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 147,
+    "resultSize" : 930,
+    "jvmGcTime" : 12,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 317390,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5463,
+  "index" : 453,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.518GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 134,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 348853,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5461,
+  "index" : 451,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.516GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 134,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 413676,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5505,
+  "index" : 495,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.616GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 133,
+    "resultSize" : 930,
+    "jvmGcTime" : 11,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 384212,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5672,
+  "index" : 662,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.016GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 131,
+    "resultSize" : 930,
+    "jvmGcTime" : 4,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 387955,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5865,
+  "index" : 855,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.514GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 131,
+    "resultSize" : 930,
+    "jvmGcTime" : 10,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 1710924,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5415,
+  "index" : 405,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.389GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 130,
+    "resultSize" : 930,
+    "jvmGcTime" : 14,
+    "resultSerializationTime" : 1,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9597,
+      "writeTime" : 377101,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5462,
+  "index" : 452,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.518GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 128,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 306075,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5411,
+  "index" : 401,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.385GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 128,
+    "resultSize" : 930,
+    "jvmGcTime" : 14,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 74684537,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5416,
+  "index" : 406,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.392GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 11,
+    "executorRunTime" : 123,
+    "resultSize" : 930,
+    "jvmGcTime" : 14,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 365240,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5924,
+  "index" : 914,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.648GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 4,
+    "executorRunTime" : 121,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 335682,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5179,
+  "index" : 169,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.866GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 114,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 276876,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5243,
+  "index" : 233,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.018GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 113,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 12003053,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5054,
+  "index" : 44,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.616GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 110,
+    "resultSize" : 930,
+    "jvmGcTime" : 22,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 299315,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5465,
+  "index" : 455,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.521GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 108,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 318366,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5614,
+  "index" : 604,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.893GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 19,
+    "executorRunTime" : 108,
+    "resultSize" : 930,
+    "jvmGcTime" : 7,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 309135,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5862,
+  "index" : 852,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.509GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 96,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 1052228,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5566,
+  "index" : 556,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.762GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 95,
+    "resultSize" : 930,
+    "jvmGcTime" : 6,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 327089,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5567,
+  "index" : 557,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.768GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 94,
+    "resultSize" : 930,
+    "jvmGcTime" : 6,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 374043,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5191,
+  "index" : 181,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.885GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 90,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 344199,
+      "recordsWritten" : 100
+    }
+  }
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation
new file mode 100644
index 0000000..63b1316
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=DECREASING_RUNTIME/json_expectation
@@ -0,0 +1,481 @@
+[ {
+  "taskId" : 5186,
+  "index" : 176,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.880GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 147,
+    "resultSize" : 930,
+    "jvmGcTime" : 12,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 317390,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5463,
+  "index" : 453,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.518GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 134,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 348853,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5461,
+  "index" : 451,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.516GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 134,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 413676,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5505,
+  "index" : 495,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.616GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 133,
+    "resultSize" : 930,
+    "jvmGcTime" : 11,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 384212,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5672,
+  "index" : 662,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.016GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 131,
+    "resultSize" : 930,
+    "jvmGcTime" : 4,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 387955,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5865,
+  "index" : 855,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.514GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 131,
+    "resultSize" : 930,
+    "jvmGcTime" : 10,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 1710924,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5415,
+  "index" : 405,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.389GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 130,
+    "resultSize" : 930,
+    "jvmGcTime" : 14,
+    "resultSerializationTime" : 1,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9597,
+      "writeTime" : 377101,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5462,
+  "index" : 452,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.518GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 128,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 306075,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5411,
+  "index" : 401,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.385GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 128,
+    "resultSize" : 930,
+    "jvmGcTime" : 14,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 74684537,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5416,
+  "index" : 406,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.392GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 11,
+    "executorRunTime" : 123,
+    "resultSize" : 930,
+    "jvmGcTime" : 14,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 365240,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5924,
+  "index" : 914,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.648GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 4,
+    "executorRunTime" : 121,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 335682,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5179,
+  "index" : 169,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.866GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 114,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 276876,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5243,
+  "index" : 233,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.018GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 113,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 12003053,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5054,
+  "index" : 44,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.616GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 110,
+    "resultSize" : 930,
+    "jvmGcTime" : 22,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 299315,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5465,
+  "index" : 455,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.521GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 108,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 318366,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5614,
+  "index" : 604,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.893GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 19,
+    "executorRunTime" : 108,
+    "resultSize" : 930,
+    "jvmGcTime" : 7,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 309135,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5862,
+  "index" : 852,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.509GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 96,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 1052228,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5566,
+  "index" : 556,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.762GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 95,
+    "resultSize" : 930,
+    "jvmGcTime" : 6,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 327089,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5567,
+  "index" : 557,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.768GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 94,
+    "resultSize" : 930,
+    "jvmGcTime" : 6,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 374043,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5191,
+  "index" : 181,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.885GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 90,
+    "resultSize" : 930,
+    "jvmGcTime" : 5,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 344199,
+      "recordsWritten" : 100
+    }
+  }
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation
new file mode 100644
index 0000000..329065b
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskList?sortBy=runtime/json_expectation
@@ -0,0 +1,481 @@
+[ {
+  "taskId" : 5610,
+  "index" : 600,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.884GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 285253,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5511,
+  "index" : 501,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.634GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 277358,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5027,
+  "index" : 17,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 289503,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5828,
+  "index" : 818,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.438GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 3,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 291997,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5173,
+  "index" : 163,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.861GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 267617,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5185,
+  "index" : 175,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.880GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 285928,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5553,
+  "index" : 543,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.719GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 298472,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5077,
+  "index" : 67,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.670GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9614,
+      "writeTime" : 259332,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5831,
+  "index" : 821,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:01.442GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9597,
+      "writeTime" : 281603,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5194,
+  "index" : 184,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.903GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9600,
+      "writeTime" : 286315,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5481,
+  "index" : 471,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.561GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9613,
+      "writeTime" : 328893,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5035,
+  "index" : 25,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.600GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 324374,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5134,
+  "index" : 124,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.782GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 279741,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5044,
+  "index" : 34,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.607GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 279851,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5026,
+  "index" : 16,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.586GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9598,
+      "writeTime" : 247775,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5331,
+  "index" : 321,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.208GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 313699,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5343,
+  "index" : 333,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.241GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 0,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9601,
+      "writeTime" : 260259,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5606,
+  "index" : 596,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:20:00.878GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 1,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9599,
+      "writeTime" : 310521,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5041,
+  "index" : 31,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.602GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 4,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9603,
+      "writeTime" : 255099,
+      "recordsWritten" : 100
+    }
+  }
+}, {
+  "taskId" : 5023,
+  "index" : 13,
+  "attempt" : 0,
+  "launchTime" : "2015-03-26T19:19:59.579GMT",
+  "executorId" : "<driver>",
+  "host" : "localhost",
+  "taskLocality" : "PROCESS_LOCAL",
+  "speculative" : false,
+  "accumulatorUpdates" : [ ],
+  "taskMetrics" : {
+    "executorDeserializeTime" : 6,
+    "executorRunTime" : 5,
+    "resultSize" : 930,
+    "jvmGcTime" : 0,
+    "resultSerializationTime" : 0,
+    "memoryBytesSpilled" : 0,
+    "diskBytesSpilled" : 0,
+    "shuffleWriteMetrics" : {
+      "bytesWritten" : 9602,
+      "writeTime" : 282108,
+      "recordsWritten" : 100
+    }
+  }
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation
new file mode 100644
index 0000000..46f32c1
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary/json_expectation
@@ -0,0 +1,15 @@
+{
+  "quantiles" : [ 0.05, 0.25, 0.5, 0.75, 0.95 ],
+  "executorDeserializeTime" : [ 0.0, 0.0, 0.0, 1.0, 3.0 ],
+  "executorRunTime" : [ 6.0, 6.0, 7.0, 10.0, 48.0 ],
+  "resultSize" : [ 930.0, 930.0, 930.0, 930.0, 930.0 ],
+  "jvmGcTime" : [ 0.0, 0.0, 0.0, 0.0, 7.0 ],
+  "resultSerializationTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
+  "memoryBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
+  "diskBytesSpilled" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ],
+  "shuffleWriteMetrics" : {
+    "writeBytes" : [ 9598.0, 9599.0, 9600.0, 9601.0, 9603.0 ],
+    "writeRecords" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ],
+    "writeTime" : [ 278805.0, 308809.0, 327953.0, 367487.0, 944783.0 ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation
new file mode 100644
index 0000000..65b4638
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1427397477963/stages/20/0/taskSummary?quantiles=0.01,0.5,0.99/json_expectation
@@ -0,0 +1,15 @@
+{
+  "quantiles" : [ 0.01, 0.5, 0.99 ],
+  "executorDeserializeTime" : [ 0.0, 0.0, 25.0 ],
+  "executorRunTime" : [ 5.0, 7.0, 123.0 ],
+  "resultSize" : [ 930.0, 930.0, 930.0 ],
+  "jvmGcTime" : [ 0.0, 0.0, 14.0 ],
+  "resultSerializationTime" : [ 0.0, 0.0, 1.0 ],
+  "memoryBytesSpilled" : [ 0.0, 0.0, 0.0 ],
+  "diskBytesSpilled" : [ 0.0, 0.0, 0.0 ],
+  "shuffleWriteMetrics" : {
+    "writeBytes" : [ 9597.0, 9600.0, 9614.0 ],
+    "writeRecords" : [ 100.0, 100.0, 100.0 ],
+    "writeTime" : [ 260991.0, 327953.0, 8980526.0 ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation
new file mode 100644
index 0000000..483632a
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-03T10:42:40.000CST/json_expectation
@@ -0,0 +1,10 @@
+[ {
+  "id" : "local-1422981759269",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-03T16:42:38.277GMT",
+    "endTime" : "2015-02-03T16:42:46.912GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation
new file mode 100644
index 0000000..4b85690
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications?maxDate=2015-02-10/json_expectation
@@ -0,0 +1,19 @@
+[ {
+  "id" : "local-1422981780767",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-03T16:42:59.720GMT",
+    "endTime" : "2015-02-03T16:43:08.731GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1422981759269",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-03T16:42:38.277GMT",
+    "endTime" : "2015-02-03T16:42:46.912GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation
new file mode 100644
index 0000000..e67f252
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications?minDate=2015-02-10/json_expectation
@@ -0,0 +1,35 @@
+[ {
+  "id" : "local-1427397477963",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-03-26T19:17:57.184GMT",
+    "endTime" : "2015-03-26T19:20:02.949GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1426533911241",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "attemptId" : "2",
+    "startTime" : "2015-03-17T23:11:50.242GMT",
+    "endTime" : "2015-03-17T23:12:25.177GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  }, {
+    "attemptId" : "1",
+    "startTime" : "2015-03-16T19:25:10.242GMT",
+    "endTime" : "2015-03-16T19:25:45.177GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1425081759269",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-28T00:02:38.277GMT",
+    "endTime" : "2015-02-28T00:02:46.912GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation
new file mode 100644
index 0000000..6101177
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation
@@ -0,0 +1,53 @@
+[ {
+  "id" : "local-1427397477963",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-03-26T19:17:57.184GMT",
+    "endTime" : "2015-03-26T19:20:02.949GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1426533911241",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "attemptId" : "2",
+    "startTime" : "2015-03-17T23:11:50.242GMT",
+    "endTime" : "2015-03-17T23:12:25.177GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  }, {
+    "attemptId" : "1",
+    "startTime" : "2015-03-16T19:25:10.242GMT",
+    "endTime" : "2015-03-16T19:25:45.177GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1425081759269",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-28T00:02:38.277GMT",
+    "endTime" : "2015-02-28T00:02:46.912GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1422981780767",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-03T16:42:59.720GMT",
+    "endTime" : "2015-02-03T16:43:08.731GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1422981759269",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-03T16:42:38.277GMT",
+    "endTime" : "2015-02-03T16:42:46.912GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation
new file mode 100644
index 0000000..8878e54
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications?status=running/json_expectation
@@ -0,0 +1 @@
+[ ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1422981759269/APPLICATION_COMPLETE
new file mode 100755
index 0000000..e69de29


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


[04/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1
new file mode 100755
index 0000000..9745b36
--- /dev/null
+++ b/core/src/test/resources/spark-events/local-1425081759269/EVENT_LOG_1
@@ -0,0 +1,88 @@
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"<driver>","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1425081759407}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1425081759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/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.7","java.vm.specification.name":"
 Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","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.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/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":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Li
 brary/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chica
 go","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/
 Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1425081759269","Timestamp":1425081758277,"User":"irashid"}
+{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\n
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.ref
 lect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"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":1425081762082,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"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":1425081762083,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"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":1425081762085,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\n
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081762069,"Completion Time":1425081762637,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n
 $line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at <console>:17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Nu
 mber of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line11.$read$$iwC$$iwC.<init>(<console>:24)\n$line11.$read$$iwC.<init>(<console>:26)\n$line11.$read.<init>(<console>:28)\n$line11.$read$.<init>(<console>:32)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\
 norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10
 .$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10
 .$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081763578,"Completion Time":1425081764005,"Accumulables":[]}}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at <console>:17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line11.$read$$iwC$$iwC.<init>(<console>:24)\n$line11.$read$$iwC.<init>(<console>:26)\n$line11.$read.<init>(<console>:28)\n$line11.$read$.<init>(<console>:32)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java
 :57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at <console>:17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line11.$read$$iwC$$iwC.<init>(<console>:24)\n$line11.$read$$iwC.<init>(<console>:26)\n$line11.$read.<init>(<console>:28)\n$line11.$read$.<init>(<console>:32)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java
 :57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764014,"Completion Time":1425081764045,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n
 $line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at <console>:19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replicatio
 n":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line13.$read$$iwC$$iwC$$iwC.<init>(<console>:24)\n$line13.$read$$iwC$$iwC.<init>(<console>:26)\n$line13.$read$$iwC.<init>(<console>:28)\n$line13.$read.<init>(<console>:30)\n$line13.$read$.<init>(<console>:34)\n$line13.$read$.<clinit>(<console>)\n$line13.$eval$.<init>(<console>:7)\n$line13.$eval$.<clinit>(<console>)\n$line13.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAc
 cessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at <console>:19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line13.$read$$iwC$$iwC$$iwC.<init>(<console>:24)\n$line13.$read$$iwC$$iwC.<init>(<console>:26)\n$line13.$read$$iwC.<init>(<console>:28)\n$line13.$read.<init>(<console>:30)\n$line13.$read$.<init>(<console>:34)\n$line13.$read$.<cl
 init>(<console>)\n$line13.$eval$.<init>(<console>:7)\n$line13.$eval$.<clinit>(<console>)\n$line13.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at <console>:19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line13.$read$$iwC$$iwC$$iwC.<init>(<console>:24)\n$line13.$read$$iwC$$iwC.<init>(<console>:26)\n$line13.$read$$iwC.<init>(<console>:28)\n$line13.$read.<init>(<console>:30)\n$line13.$read$.<init>(<console>:34)\n$line13.$read$.<cl
 init>(<console>)\n$line13.$eval$.<init>(<console>:7)\n$line13.$eval$.<clinit>(<console>)\n$line13.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764396,"Completion Time":1425081764648,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line15.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line15.$read$$iwC$$iwC.<init>(<console>:24)\n$line15.$read$$iwC.<init>(<console>:26)\n$line15.$read.<init
 >(<console>:28)\n$line15.$read$.<init>(<console>:32)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.<init>(<console>:7)\n$line15.$eval$.<clinit>(<console>)\n$line15.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line15.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line15.$read$$iwC$$iwC.<init>(<console>:24)\n$line15.$read$$iwC.<init>(<console>:26)\n$line15.$read.<init>(<cons
 ole>:28)\n$line15.$read$.<init>(<console>:32)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.<init>(<console>:7)\n$line15.$eval$.<clinit>(<console>)\n$line15.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line15.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line15.$read$$iwC$$iwC.<init>(<console>:24)\n$line15.$read$$iwC.<init>(<console>:26)\n$line15.$read.<init>(<cons
 ole>:28)\n$line15.$read$.<init>(<console>:32)\n$line15.$read$.<clinit>(<console>)\n$line15.$eval$.<init>(<console>:7)\n$line15.$eval$.<clinit>(<console>)\n$line15.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081765026,"Completion Time":1425081765050,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerApplicationEnd","Timestamp":1425081766912}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1425081759269/SPARK_VERSION_1.2.0
new file mode 100755
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE
new file mode 100755
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1
new file mode 100755
index 0000000..9ef5bd5
--- /dev/null
+++ b/core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1
@@ -0,0 +1,24 @@
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"<driver>","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426533911361}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426533911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/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/irashid","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_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/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.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Li
 brary/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chica
 go","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/
 Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426533910242,"User":"irashid","App Attempt ID":"1"}
+{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$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:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\ns
 un.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"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":1426533936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"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":1426533936522,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"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":1426533936524,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\ns
 un.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426533936103,"Completion Time":1426533936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}}
+{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerApplicationEnd","Timestamp":1426533945177}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0
new file mode 100755
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1426633911242/APPLICATION_COMPLETE
new file mode 100755
index 0000000..e69de29


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


[05/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1 b/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1
new file mode 100755
index 0000000..f14a000
--- /dev/null
+++ b/core/src/test/resources/spark-events/local-1422981780767/EVENT_LOG_1
@@ -0,0 +1,82 @@
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"<driver>","Host":"localhost","Port":57971},"Maximum Memory":278302556,"Timestamp":1422981780906}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57969","spark.repl.class.uri":"http://192.168.1.103:57968","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57970","spark.tachyonStore.folderName":"spark-3f19daee-844c-41d0-a3fc-5e3e508f9731","spark.app.id":"local-1422981780767"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/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.7","java.vm.specification.name":"
 Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","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.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/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":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Li
 brary/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chica
 go","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/
 Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981780767","Timestamp":1422981779720,"User":"irashid"}
+{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\n
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.ref
 lect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"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":1422981784243,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":541,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"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":1422981784241,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at <console>:15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\n
 sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981784228,"Completion Time":1422981784819,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n
 $line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at <console>:20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Nu
 mber of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(D
 elegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10
 .$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":88000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":94000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":79000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":73000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":83000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":436,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":98000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786285,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":101000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":76000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at <console>:14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10
 .$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981785829,"Completion Time":1422981786286,"Accumulables":[]}}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at <console>:20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$
 line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"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":8,"Fetch Wait Time":1,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"got a 3, failing","Stack Trace":[{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":18},{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"<console>","Line Number":17},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":328},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1311},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},
 {"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":61},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":56},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":196},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1145},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":615},{"Declaring Class":"java.lang.Thread
 ","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecuto
 r.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n","Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"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":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786343,"Failed":true,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"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":8,"Fe
 tch Wait Time":0,"Remote Bytes Read":0}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at <console>:20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$
 line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981786296,"Completion Time":1422981786347,"Failure Reason":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, fai
 ling\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Th
 read.java:745)\n\nDriver stacktrace:","Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobFailed","Exception":{"Message":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:5
 6)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Stack Trace":[{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages","File Name":"DAGScheduler.scala","Line Number":1214},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1203},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"scala.collection.mutable.ResizableArray$class","Method Name":"foreach","File Name":"ResizableArray.scala","Line Number":59},{"Dec
 laring Class":"scala.collection.mutable.ArrayBuffer","Method Name":"foreach","File Name":"ArrayBuffer.scala","Line Number":47},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"abortStage","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"scala.Option","Method Name":"foreach","File Name":"Option.scala","Line Number":236},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"handleTaskSetFailed","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2","Method Name":"applyOrElse","File Name":"DAGSch
 eduler.scala","Line Number":1420},{"Declaring Class":"akka.actor.Actor$class","Method Name":"aroundReceive","File Name":"Actor.scala","Line Number":465},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor","Method Name":"aroundReceive","File Name":"DAGScheduler.scala","Line Number":1375},{"Declaring Class":"akka.actor.ActorCell","Method Name":"receiveMessage","File Name":"ActorCell.scala","Line Number":516},{"Declaring Class":"akka.actor.ActorCell","Method Name":"invoke","File Name":"ActorCell.scala","Line Number":487},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"processMailbox","File Name":"Mailbox.scala","Line Number":238},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"run","File Name":"Mailbox.scala","Line Number":220},{"Declaring Class":"akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask","Method Name":"exec","File Name":"AbstractDispatcher.scala","Line Number":393},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinTa
 sk","Method Name":"doExec","File Name":"ForkJoinTask.java","Line Number":260},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool$WorkQueue","Method Name":"runTask","File Name":"ForkJoinPool.java","Line Number":1339},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool","Method Name":"runWorker","File Name":"ForkJoinPool.java","Line Number":1979},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinWorkerThread","Method Name":"run","File Name":"ForkJoinWorkerThread.java","Line Number":107}]}}}
+{"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line19.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line19.$read$$iwC$$iwC.<init>(<console>:24)\n$line19.$read$$iwC.<init>(<console>:26)\n$line19.$read.<
 init>(<console>:28)\n$line19.$read$.<init>(<console>:32)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.<init>(<console>:7)\n$line19.$eval$.<clinit>(<console>)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3]}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line19.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line19.$read$$iwC$$iwC.<init>(<console>:24)\n$line19.$read$$iwC.<init>(<console>:26)\n$line19.$read.<init>(<
 console>:28)\n$line19.$read$.<init>(<console>:32)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.<init>(<console>:7)\n$line19.$eval$.<clinit>(<console>)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"<driver>","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":21,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at <console>:17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line19.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line19.$read$$iwC$$iwC.<init>(<console>:24)\n$line19.$read$$iwC.<init>(<console>:26)\n$line19.$read.<init>(<
 console>:28)\n$line19.$read$.<init>(<console>:32)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.<init>(<console>:7)\n$line19.$eval$.<clinit>(<console>)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981787191,"Completion Time":1422981787226,"Accumulables":[]}}
+{"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerApplicationEnd","Timestamp":1422981788731}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0 b/core/src/test/resources/spark-events/local-1422981780767/SPARK_VERSION_1.2.0
new file mode 100755
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE
----------------------------------------------------------------------
diff --git a/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE b/core/src/test/resources/spark-events/local-1425081759269/APPLICATION_COMPLETE
new file mode 100755
index 0000000..e69de29


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


[09/10] spark git commit: [SPARK-3454] separate json endpoints for data in the UI

Posted by ir...@apache.org.
http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala
new file mode 100644
index 0000000..07b224f
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneRDDResource.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.{PathParam, GET, Produces}
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.ui.SparkUI
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneRDDResource(ui: SparkUI) {
+
+  @GET
+  def rddData(@PathParam("rddId") rddId: Int): RDDStorageInfo  = {
+    AllRDDResource.getRDDStorageInfo(rddId, ui.storageListener, true).getOrElse(
+      throw new NotFoundException(s"no rdd found w/ id $rddId")
+    )
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala
new file mode 100644
index 0000000..fd24aea
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneStageResource.scala
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.SparkException
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.status.api.v1.StageStatus._
+import org.apache.spark.status.api.v1.TaskSorting._
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.ui.jobs.JobProgressListener
+import org.apache.spark.ui.jobs.UIData.StageUIData
+
+@Produces(Array(MediaType.APPLICATION_JSON))
+private[v1] class OneStageResource(ui: SparkUI) {
+
+  @GET
+  @Path("")
+  def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = {
+    withStage(stageId){ stageAttempts =>
+      stageAttempts.map { stage =>
+        AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui,
+          includeDetails = true)
+      }
+    }
+  }
+
+  @GET
+  @Path("/{stageAttemptId: \\d+}")
+  def oneAttemptData(
+      @PathParam("stageId") stageId: Int,
+      @PathParam("stageAttemptId") stageAttemptId: Int): StageData = {
+    withStageAttempt(stageId, stageAttemptId) { stage =>
+      AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui,
+        includeDetails = true)
+    }
+  }
+
+  @GET
+  @Path("/{stageAttemptId: \\d+}/taskSummary")
+  def taskSummary(
+      @PathParam("stageId") stageId: Int,
+      @PathParam("stageAttemptId") stageAttemptId: Int,
+      @DefaultValue("0.05,0.25,0.5,0.75,0.95") @QueryParam("quantiles") quantileString: String)
+  : TaskMetricDistributions = {
+    withStageAttempt(stageId, stageAttemptId) { stage =>
+      val quantiles = quantileString.split(",").map { s =>
+        try {
+          s.toDouble
+        } catch {
+          case nfe: NumberFormatException =>
+            throw new BadParameterException("quantiles", "double", s)
+        }
+      }
+      AllStagesResource.taskMetricDistributions(stage.ui.taskData.values, quantiles)
+    }
+  }
+
+  @GET
+  @Path("/{stageAttemptId: \\d+}/taskList")
+  def taskList(
+      @PathParam("stageId") stageId: Int,
+      @PathParam("stageAttemptId") stageAttemptId: Int,
+      @DefaultValue("0") @QueryParam("offset") offset: Int,
+      @DefaultValue("20") @QueryParam("length") length: Int,
+      @DefaultValue("ID") @QueryParam("sortBy") sortBy: TaskSorting): Seq[TaskData] = {
+    withStageAttempt(stageId, stageAttemptId) { stage =>
+      val tasks = stage.ui.taskData.values.map{AllStagesResource.convertTaskData}.toIndexedSeq
+        .sorted(OneStageResource.ordering(sortBy))
+      tasks.slice(offset, offset + length)  
+    }
+  }
+
+  private case class StageStatusInfoUi(status: StageStatus, info: StageInfo, ui: StageUIData)
+
+  private def withStage[T](stageId: Int)(f: Seq[StageStatusInfoUi] => T): T = {
+    val stageAttempts = findStageStatusUIData(ui.jobProgressListener, stageId)
+    if (stageAttempts.isEmpty) {
+      throw new NotFoundException("unknown stage: " + stageId)
+    } else {
+      f(stageAttempts)
+    }
+  }
+
+  private def findStageStatusUIData(
+      listener: JobProgressListener,
+      stageId: Int): Seq[StageStatusInfoUi] = {
+    listener.synchronized {
+      def getStatusInfoUi(status: StageStatus, infos: Seq[StageInfo]): Seq[StageStatusInfoUi] = {
+        infos.filter { _.stageId == stageId }.map { info =>
+          val ui = listener.stageIdToData.getOrElse((info.stageId, info.attemptId),
+            // this is an internal error -- we should always have uiData
+            throw new SparkException(
+              s"no stage ui data found for stage: ${info.stageId}:${info.attemptId}")
+          )
+          StageStatusInfoUi(status, info, ui)
+        }
+      }
+      getStatusInfoUi(ACTIVE, listener.activeStages.values.toSeq) ++
+        getStatusInfoUi(COMPLETE, listener.completedStages) ++
+        getStatusInfoUi(FAILED, listener.failedStages) ++
+        getStatusInfoUi(PENDING, listener.pendingStages.values.toSeq)
+    }
+  }
+
+  private def withStageAttempt[T](
+      stageId: Int,
+      stageAttemptId: Int)
+      (f: StageStatusInfoUi => T): T = {
+    withStage(stageId) { attempts =>
+        val oneAttempt = attempts.find { stage => stage.info.attemptId == stageAttemptId }
+        oneAttempt match {
+          case Some(stage) =>
+            f(stage)
+          case None =>
+            val stageAttempts = attempts.map { _.info.attemptId }
+            throw new NotFoundException(s"unknown attempt for stage $stageId.  " +
+              s"Found attempts: ${stageAttempts.mkString("[", ",", "]")}")
+        }
+    }
+  }
+}
+
+object OneStageResource {
+  def ordering(taskSorting: TaskSorting): Ordering[TaskData] = {
+    val extractor: (TaskData => Long) = td =>
+      taskSorting match {
+        case ID => td.taskId
+        case INCREASING_RUNTIME => td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L)
+        case DECREASING_RUNTIME => -td.taskMetrics.map{_.executorRunTime}.getOrElse(-1L)
+      }
+    Ordering.by(extractor)
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala
new file mode 100644
index 0000000..95fbd96
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import javax.ws.rs.WebApplicationException
+import javax.ws.rs.core.Response
+
+import com.sun.jersey.spi.container.{ContainerRequest, ContainerRequestFilter}
+
+private[v1] class SecurityFilter extends ContainerRequestFilter with UIRootFromServletContext {
+  def filter(req: ContainerRequest): ContainerRequest = {
+    val user = Option(req.getUserPrincipal).map { _.getName }.orNull
+    if (uiRoot.securityManager.checkUIViewPermissions(user)) {
+      req
+    } else {
+      throw new WebApplicationException(
+        Response
+          .status(Response.Status.FORBIDDEN)
+          .entity(raw"""user "$user"is not authorized""")
+          .build()
+      )
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala
new file mode 100644
index 0000000..cee2978
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.text.SimpleDateFormat
+import java.util.TimeZone
+import javax.ws.rs.WebApplicationException
+import javax.ws.rs.core.Response
+import javax.ws.rs.core.Response.Status
+
+import scala.util.Try
+
+private[v1] class SimpleDateParam(val originalValue: String) {
+  val timestamp: Long = {
+    SimpleDateParam.formats.collectFirst {
+      case fmt if Try(fmt.parse(originalValue)).isSuccess =>
+        fmt.parse(originalValue).getTime()
+    }.getOrElse(
+      throw new WebApplicationException(
+        Response
+          .status(Status.BAD_REQUEST)
+          .entity("Couldn't parse date: " + originalValue)
+          .build()
+      )
+    )
+  }
+}
+
+private[v1] object SimpleDateParam {
+
+  val formats: Seq[SimpleDateFormat] = {
+
+    val gmtDay = new SimpleDateFormat("yyyy-MM-dd")
+    gmtDay.setTimeZone(TimeZone.getTimeZone("GMT"))
+
+    Seq(
+      new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz"),
+      gmtDay
+    )
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/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
new file mode 100644
index 0000000..ef3c857
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.status.api.v1
+
+import java.util.Date
+
+import scala.collection.Map
+
+import org.apache.spark.JobExecutionStatus
+
+class ApplicationInfo private[spark](
+    val id: String,
+    val name: String,
+    val attempts: Seq[ApplicationAttemptInfo])
+
+class ApplicationAttemptInfo private[spark](
+    val attemptId: Option[String],
+    val startTime: Date,
+    val endTime: Date,
+    val sparkUser: String,
+    val completed: Boolean = false)
+
+class ExecutorStageSummary private[spark](
+    val taskTime : Long,
+    val failedTasks : Int,
+    val succeededTasks : Int,
+    val inputBytes : Long,
+    val outputBytes : Long,
+    val shuffleRead : Long,
+    val shuffleWrite : Long,
+    val memoryBytesSpilled : Long,
+    val diskBytesSpilled : Long)
+
+class ExecutorSummary private[spark](
+    val id: String,
+    val hostPort: String,
+    val rddBlocks: Int,
+    val memoryUsed: Long,
+    val diskUsed: Long,
+    val activeTasks: Int,
+    val failedTasks: Int,
+    val completedTasks: Int,
+    val totalTasks: Int,
+    val totalDuration: Long,
+    val totalInputBytes: Long,
+    val totalShuffleRead: Long,
+    val totalShuffleWrite: Long,
+    val maxMemory: Long,
+    val executorLogs: Map[String, String])
+
+class JobData private[spark](
+    val jobId: Int,
+    val name: String,
+    val description: Option[String],
+    val submissionTime: Option[Date],
+    val completionTime: Option[Date],
+    val stageIds: Seq[Int],
+    val jobGroup: Option[String],
+    val status: JobExecutionStatus,
+    val numTasks: Int,
+    val numActiveTasks: Int,
+    val numCompletedTasks: Int,
+    val numSkippedTasks: Int,
+    val numFailedTasks: Int,
+    val numActiveStages: Int,
+    val numCompletedStages: Int,
+    val numSkippedStages: Int,
+    val numFailedStages: Int)
+
+// Q: should Tachyon size go in here as well?  currently the UI only shows it on the overall storage
+// page ... does anybody pay attention to it?
+class RDDStorageInfo private[spark](
+    val id: Int,
+    val name: String,
+    val numPartitions: Int,
+    val numCachedPartitions: Int,
+    val storageLevel: String,
+    val memoryUsed: Long,
+    val diskUsed: Long,
+    val dataDistribution: Option[Seq[RDDDataDistribution]],
+    val partitions: Option[Seq[RDDPartitionInfo]])
+
+class RDDDataDistribution private[spark](
+    val address: String,
+    val memoryUsed: Long,
+    val memoryRemaining: Long,
+    val diskUsed: Long)
+
+class RDDPartitionInfo private[spark](
+    val blockName: String,
+    val storageLevel: String,
+    val memoryUsed: Long,
+    val diskUsed: Long,
+    val executors: Seq[String])
+
+class StageData private[spark](
+    val status: StageStatus,
+    val stageId: Int,
+    val attemptId: Int,
+    val numActiveTasks: Int ,
+    val numCompleteTasks: Int,
+    val numFailedTasks: Int,
+
+    val executorRunTime: Long,
+
+    val inputBytes: Long,
+    val inputRecords: Long,
+    val outputBytes: Long,
+    val outputRecords: Long,
+    val shuffleReadBytes: Long,
+    val shuffleReadRecords: Long,
+    val shuffleWriteBytes: Long,
+    val shuffleWriteRecords: Long,
+    val memoryBytesSpilled: Long,
+    val diskBytesSpilled: Long,
+
+    val name: String,
+    val details: String,
+    val schedulingPool: String,
+
+    val accumulatorUpdates: Seq[AccumulableInfo],
+    val tasks: Option[Map[Long, TaskData]],
+    val executorSummary:Option[Map[String,ExecutorStageSummary]])
+
+class TaskData private[spark](
+    val taskId: Long,
+    val index: Int,
+    val attempt: Int,
+    val launchTime: Date,
+    val executorId: String,
+    val host: String,
+    val taskLocality: String,
+    val speculative: Boolean,
+    val accumulatorUpdates: Seq[AccumulableInfo],
+    val errorMessage: Option[String] = None,
+    val taskMetrics: Option[TaskMetrics] = None)
+
+class TaskMetrics private[spark](
+    val executorDeserializeTime: Long,
+    val executorRunTime: Long,
+    val resultSize: Long,
+    val jvmGcTime: Long,
+    val resultSerializationTime: Long,
+    val memoryBytesSpilled: Long,
+    val diskBytesSpilled: Long,
+    val inputMetrics: Option[InputMetrics],
+    val outputMetrics: Option[OutputMetrics],
+    val shuffleReadMetrics: Option[ShuffleReadMetrics],
+    val shuffleWriteMetrics: Option[ShuffleWriteMetrics])
+
+class InputMetrics private[spark](
+    val bytesRead: Long,
+    val recordsRead: Long)
+
+class OutputMetrics private[spark](
+    val bytesWritten: Long,
+    val recordsWritten: Long)
+
+class ShuffleReadMetrics private[spark](
+    val remoteBlocksFetched: Int,
+    val localBlocksFetched: Int,
+    val fetchWaitTime: Long,
+    val remoteBytesRead: Long,
+    val totalBlocksFetched: Int,
+    val recordsRead: Long)
+
+class ShuffleWriteMetrics private[spark](
+    val bytesWritten: Long,
+    val writeTime: Long,
+    val recordsWritten: Long)
+
+class TaskMetricDistributions private[spark](
+    val quantiles: IndexedSeq[Double],
+
+    val executorDeserializeTime: IndexedSeq[Double],
+    val executorRunTime: IndexedSeq[Double],
+    val resultSize: IndexedSeq[Double],
+    val jvmGcTime: IndexedSeq[Double],
+    val resultSerializationTime: IndexedSeq[Double],
+    val memoryBytesSpilled: IndexedSeq[Double],
+    val diskBytesSpilled: IndexedSeq[Double],
+
+    val inputMetrics: Option[InputMetricDistributions],
+    val outputMetrics: Option[OutputMetricDistributions],
+    val shuffleReadMetrics: Option[ShuffleReadMetricDistributions],
+    val shuffleWriteMetrics: Option[ShuffleWriteMetricDistributions])
+
+class InputMetricDistributions private[spark](
+    val bytesRead: IndexedSeq[Double],
+    val recordsRead: IndexedSeq[Double])
+
+class OutputMetricDistributions private[spark](
+    val bytesWritten: IndexedSeq[Double],
+    val recordsWritten: IndexedSeq[Double])
+
+class ShuffleReadMetricDistributions private[spark](
+    val readBytes: IndexedSeq[Double],
+    val readRecords: IndexedSeq[Double],
+    val remoteBlocksFetched: IndexedSeq[Double],
+    val localBlocksFetched: IndexedSeq[Double],
+    val fetchWaitTime: IndexedSeq[Double],
+    val remoteBytesRead: IndexedSeq[Double],
+    val totalBlocksFetched: IndexedSeq[Double])
+
+class ShuffleWriteMetricDistributions private[spark](
+    val writeBytes: IndexedSeq[Double],
+    val writeRecords: IndexedSeq[Double],
+    val writeTime: IndexedSeq[Double])
+
+class AccumulableInfo private[spark](
+    val id: Long,
+    val name: String,
+    val update: Option[String],
+    val value: String)

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
index 7d75929..ec71148 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
@@ -25,13 +25,17 @@ import org.apache.spark.scheduler._
 /**
  * :: DeveloperApi ::
  * A SparkListener that maintains executor storage status.
+ *
+ * This class is thread-safe (unlike JobProgressListener)
  */
 @DeveloperApi
 class StorageStatusListener extends SparkListener {
   // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE)
   private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]()
 
-  def storageStatusList: Seq[StorageStatus] = executorIdToStorageStatus.values.toSeq
+  def storageStatusList: Seq[StorageStatus] = synchronized {
+    executorIdToStorageStatus.values.toSeq
+  }
 
   /** Update storage status list to reflect updated block statuses */
   private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) {

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index a5271f0..bfe4a18 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -17,6 +17,9 @@
 
 package org.apache.spark.ui
 
+import java.util.Date
+
+import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo, JsonRootResource, UIRoot}
 import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext}
 import org.apache.spark.scheduler._
 import org.apache.spark.storage.StorageStatusListener
@@ -33,7 +36,7 @@ import org.apache.spark.ui.scope.RDDOperationGraphListener
 private[spark] class SparkUI private (
     val sc: Option[SparkContext],
     val conf: SparkConf,
-    val securityManager: SecurityManager,
+    securityManager: SecurityManager,
     val environmentListener: EnvironmentListener,
     val storageStatusListener: StorageStatusListener,
     val executorsListener: ExecutorsListener,
@@ -41,22 +44,27 @@ private[spark] class SparkUI private (
     val storageListener: StorageListener,
     val operationGraphListener: RDDOperationGraphListener,
     var appName: String,
-    val basePath: String)
+    val basePath: String,
+    val startTime: Long)
   extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI")
-  with Logging {
+  with Logging
+  with UIRoot {
 
   val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false)
 
+
+  val stagesTab = new StagesTab(this)
+
   /** Initialize all components of the server. */
   def initialize() {
     attachTab(new JobsTab(this))
-    val stagesTab = new StagesTab(this)
     attachTab(stagesTab)
     attachTab(new StorageTab(this))
     attachTab(new EnvironmentTab(this))
     attachTab(new ExecutorsTab(this))
     attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
     attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath))
+    attachHandler(JsonRootResource.getJsonServlet(this))
     // This should be POST only, but, the YARN AM proxy won't proxy POSTs
     attachHandler(createRedirectHandler(
       "/stages/stage/kill", "/stages", stagesTab.handleKillRequest,
@@ -83,6 +91,24 @@ private[spark] class SparkUI private (
   private[spark] def appUIHostPort = publicHostName + ":" + boundPort
 
   private[spark] def appUIAddress = s"http://$appUIHostPort"
+
+  def getSparkUI(appId: String): Option[SparkUI] = {
+    if (appId == appName) Some(this) else None
+  }
+
+  def getApplicationInfoList: Iterator[ApplicationInfo] = {
+    Iterator(new ApplicationInfo(
+      id = appName,
+      name = appName,
+      attempts = Seq(new ApplicationAttemptInfo(
+        attemptId = None,
+        startTime = new Date(startTime),
+        endTime = new Date(-1),
+        sparkUser = "",
+        completed = false
+      ))
+    ))
+  }
 }
 
 private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String)
@@ -109,9 +135,10 @@ private[spark] object SparkUI {
       listenerBus: SparkListenerBus,
       jobProgressListener: JobProgressListener,
       securityManager: SecurityManager,
-      appName: String): SparkUI =  {
+      appName: String,
+      startTime: Long): SparkUI =  {
     create(Some(sc), conf, listenerBus, securityManager, appName,
-      jobProgressListener = Some(jobProgressListener))
+      jobProgressListener = Some(jobProgressListener), startTime = startTime)
   }
 
   def createHistoryUI(
@@ -119,8 +146,9 @@ private[spark] object SparkUI {
       listenerBus: SparkListenerBus,
       securityManager: SecurityManager,
       appName: String,
-      basePath: String): SparkUI = {
-    create(None, conf, listenerBus, securityManager, appName, basePath)
+      basePath: String,
+      startTime: Long): SparkUI = {
+    create(None, conf, listenerBus, securityManager, appName, basePath, startTime = startTime)
   }
 
   /**
@@ -137,7 +165,8 @@ private[spark] object SparkUI {
       securityManager: SecurityManager,
       appName: String,
       basePath: String = "",
-      jobProgressListener: Option[JobProgressListener] = None): SparkUI = {
+      jobProgressListener: Option[JobProgressListener] = None,
+      startTime: Long): SparkUI = {
 
     val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse {
       val listener = new JobProgressListener(conf)
@@ -159,6 +188,6 @@ private[spark] object SparkUI {
 
     new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener,
       executorsListener, _jobProgressListener, storageListener, operationGraphListener,
-      appName, basePath)
+      appName, basePath, startTime)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/WebUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
index f9860d1..384f2ad 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -37,7 +37,7 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf}
  * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly.
  */
 private[spark] abstract class WebUI(
-    securityManager: SecurityManager,
+    val securityManager: SecurityManager,
     port: Int,
     conf: SparkConf,
     basePath: String = "",
@@ -77,15 +77,9 @@ private[spark] abstract class WebUI(
     val pagePath = "/" + page.prefix
     val renderHandler = createServletHandler(pagePath,
       (request: HttpServletRequest) => page.render(request), securityManager, basePath)
-    val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json",
-      (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)
     attachHandler(renderHandler)
-    attachHandler(renderJsonHandler)
     pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
       .append(renderHandler)
-    pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
-      .append(renderJsonHandler)
-    
   }
 
   /** Attach a handler to this UI. */

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/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 956608d..b247e4c 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
@@ -22,11 +22,11 @@ import javax.servlet.http.HttpServletRequest
 
 import scala.xml.Node
 
+import org.apache.spark.status.api.v1.ExecutorSummary
 import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage}
 import org.apache.spark.util.Utils
 
-/** Summary information about an executor to display in the UI. */
-// Needs to be private[ui] because of a false positive MiMa failure.
+// This isn't even used anymore -- but we need to keep it b/c of a MiMa false positive
 private[ui] case class ExecutorSummaryInfo(
     id: String,
     hostPort: String,
@@ -44,6 +44,7 @@ private[ui] case class ExecutorSummaryInfo(
     maxMemory: Long,
     executorLogs: Map[String, String])
 
+
 private[ui] class ExecutorsPage(
     parent: ExecutorsTab,
     threadDumpEnabled: Boolean)
@@ -55,7 +56,8 @@ private[ui] class ExecutorsPage(
     val maxMem = storageStatusList.map(_.maxMem).sum
     val memUsed = storageStatusList.map(_.memUsed).sum
     val diskUsed = storageStatusList.map(_.diskUsed).sum
-    val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId)
+    val execInfo = for (statusId <- 0 until storageStatusList.size) yield
+      ExecutorsPage.getExecInfo(listener, statusId)
     val execInfoSorted = execInfo.sortBy(_.id)
     val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty
 
@@ -111,7 +113,7 @@ private[ui] class ExecutorsPage(
   }
 
   /** Render an HTML row representing an executor */
-  private def execRow(info: ExecutorSummaryInfo, logsExist: Boolean): Seq[Node] = {
+  private def execRow(info: ExecutorSummary, logsExist: Boolean): Seq[Node] = {
     val maximumMemory = info.maxMemory
     val memoryUsed = info.memoryUsed
     val diskUsed = info.diskUsed
@@ -170,8 +172,11 @@ private[ui] class ExecutorsPage(
     </tr>
   }
 
+}
+
+private[spark] object ExecutorsPage {
   /** Represent an executor's info as a map given a storage status index */
-  private def getExecInfo(statusId: Int): ExecutorSummaryInfo = {
+  def getExecInfo(listener: ExecutorsListener, statusId: Int): ExecutorSummary = {
     val status = listener.storageStatusList(statusId)
     val execId = status.blockManagerId.executorId
     val hostPort = status.blockManagerId.hostPort
@@ -189,7 +194,7 @@ private[ui] class ExecutorsPage(
     val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L)
     val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty)
 
-    new ExecutorSummaryInfo(
+    new ExecutorSummary(
       execId,
       hostPort,
       rddBlocks,

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
index f6abf27..09323d1 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
@@ -271,6 +271,12 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
       val shouldShowCompletedJobs = completedJobs.nonEmpty
       val shouldShowFailedJobs = failedJobs.nonEmpty
 
+      val completedJobNumStr = if (completedJobs.size == listener.numCompletedJobs) {
+        s"${completedJobs.size}"
+      } else {
+        s"${listener.numCompletedJobs}, only showing ${completedJobs.size}"
+      }
+
       val summary: NodeSeq =
         <div>
           <ul class="unstyled">
@@ -295,9 +301,9 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
             }
             {
               if (shouldShowCompletedJobs) {
-                <li>
+                <li id="completed-summary">
                   <a href="#completed"><strong>Completed Jobs:</strong></a>
-                  {completedJobs.size}
+                  {completedJobNumStr}
                 </li>
               }
             }
@@ -305,7 +311,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
               if (shouldShowFailedJobs) {
                 <li>
                   <a href="#failed"><strong>Failed Jobs:</strong></a>
-                  {failedJobs.size}
+                  {listener.numFailedJobs}
                 </li>
               }
             }
@@ -322,7 +328,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
           activeJobsTable
       }
       if (shouldShowCompletedJobs) {
-        content ++= <h4 id="completed">Completed Jobs ({completedJobs.size})</h4> ++
+        content ++= <h4 id="completed">Completed Jobs ({completedJobNumStr})</h4> ++
           completedJobsTable
       }
       if (shouldShowFailedJobs) {

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
index 236bc8e..a37f739 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
@@ -64,6 +64,12 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
       val shouldShowCompletedStages = completedStages.nonEmpty
       val shouldShowFailedStages = failedStages.nonEmpty
 
+      val completedStageNumStr = if (numCompletedStages == completedStages.size) {
+        s"$numCompletedStages"
+      } else {
+        s"$numCompletedStages, only showing ${completedStages.size}"
+      }
+
       val summary: NodeSeq =
         <div>
           <ul class="unstyled">
@@ -98,9 +104,9 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
             }
             {
               if (shouldShowCompletedStages) {
-                <li>
+                <li id="completed-summary">
                   <a href="#completed"><strong>Completed Stages:</strong></a>
-                  {numCompletedStages}
+                  {completedStageNumStr}
                 </li>
               }
             }
@@ -132,7 +138,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
         pendingStagesTable.toNodeSeq
       }
       if (shouldShowCompletedStages) {
-        content ++= <h4 id="completed">Completed Stages ({numCompletedStages})</h4> ++
+        content ++= <h4 id="completed">Completed Stages ({completedStageNumStr})</h4> ++
         completedStagesTable.toNodeSeq
       }
       if (shouldShowFailedStages) {

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
index 96cc3d7..7163217 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
@@ -187,7 +187,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") {
       val jobDataOption = listener.jobIdToData.get(jobId)
       if (jobDataOption.isEmpty) {
         val content =
-          <div>
+          <div id="no-info">
             <p>No information to display for job {jobId}</p>
           </div>
         return UIUtils.headerSparkPage(

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index 8f9aa9f..246e191 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -74,6 +74,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
   // JobProgressListener's retention limits.
   var numCompletedStages = 0
   var numFailedStages = 0
+  var numCompletedJobs = 0
+  var numFailedJobs = 0
 
   // Misc:
   val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]()
@@ -217,10 +219,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
         completedJobs += jobData
         trimJobsIfNecessary(completedJobs)
         jobData.status = JobExecutionStatus.SUCCEEDED
+        numCompletedJobs += 1
       case JobFailed(exception) =>
         failedJobs += jobData
         trimJobsIfNecessary(failedJobs)
         jobData.status = JobExecutionStatus.FAILED
+        numFailedJobs += 1
     }
     for (stageId <- jobData.stageIds) {
       stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage =>

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
index d725b9d..f3e0b38 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
@@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest
 
 import scala.xml.Node
 
-import org.apache.spark.scheduler.{Schedulable, StageInfo}
+import org.apache.spark.scheduler.StageInfo
 import org.apache.spark.ui.{WebUIPage, UIUtils}
 
 /** Page showing specific pool details */

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 5793100..89d175b 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -25,11 +25,11 @@ import scala.xml.{Elem, Node, Unparsed}
 import org.apache.commons.lang3.StringEscapeUtils
 
 import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo}
 import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils}
 import org.apache.spark.ui.jobs.UIData._
 import org.apache.spark.ui.scope.RDDOperationGraph
 import org.apache.spark.util.{Utils, Distribution}
-import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo}
 
 /** Page showing statistics and task list for a given stage */
 private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
@@ -48,14 +48,22 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
       val stageAttemptId = parameterAttempt.toInt
       val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId))
 
-      if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) {
+      val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)"
+      if (stageDataOption.isEmpty) {
+        val content =
+          <div id="no-info">
+            <p>No information to display for Stage {stageId} (Attempt {stageAttemptId})</p>
+          </div>
+        return UIUtils.headerSparkPage(stageHeader, content, parent)
+
+      }
+      if (stageDataOption.get.taskData.isEmpty) {
         val content =
           <div>
             <h4>Summary Metrics</h4> No tasks have started yet
             <h4>Tasks</h4> No tasks have started yet
           </div>
-        return UIUtils.headerSparkPage(
-          s"Details for Stage $stageId (Attempt $stageAttemptId)", content, parent)
+        return UIUtils.headerSparkPage(stageHeader, content, parent)
       }
 
       val stageData = stageDataOption.get
@@ -446,8 +454,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
         maybeAccumulableTable ++
         <h4>Tasks</h4> ++ taskTable
 
-      UIUtils.headerSparkPage(
-        "Details for Stage %d".format(stageId), content, parent, showVisualization = true)
+      UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
index 199f731..05f94a7 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
@@ -21,8 +21,8 @@ import javax.servlet.http.HttpServletRequest
 
 import scala.xml.Node
 
-import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils}
-import org.apache.spark.ui.{WebUIPage, UIUtils}
+import org.apache.spark.status.api.v1.{AllRDDResource, RDDDataDistribution, RDDPartitionInfo}
+import org.apache.spark.ui.{UIUtils, WebUIPage}
 import org.apache.spark.util.Utils
 
 /** Page showing storage details for a given RDD */
@@ -32,28 +32,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
   def render(request: HttpServletRequest): Seq[Node] = {
     val parameterId = request.getParameter("id")
     require(parameterId != null && parameterId.nonEmpty, "Missing id parameter")
-
     val rddId = parameterId.toInt
-    val storageStatusList = listener.storageStatusList
-    val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse {
-      // Rather than crashing, render an "RDD Not Found" page
-      return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent)
-    }
+    val rddStorageInfo = AllRDDResource.getRDDStorageInfo(rddId, listener,includeDetails = true)
+      .getOrElse {
+        // Rather than crashing, render an "RDD Not Found" page
+        return UIUtils.headerSparkPage("RDD Not Found", Seq[Node](), parent)
+      }
 
     // Worker table
-    val workers = storageStatusList.map((rddId, _))
-    val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers,
-      id = Some("rdd-storage-by-worker-table"))
+    val workerTable = UIUtils.listingTable(workerHeader, workerRow,
+      rddStorageInfo.dataDistribution.get, id = Some("rdd-storage-by-worker-table"))
 
     // Block table
-    val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList)
-    val blocks = storageStatusList
-      .flatMap(_.rddBlocksById(rddId))
-      .sortWith(_._1.name < _._1.name)
-      .map { case (blockId, status) =>
-        (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown")))
-      }
-    val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks,
+    val blockTable = UIUtils.listingTable(blockHeader, blockRow, rddStorageInfo.partitions.get,
       id = Some("rdd-storage-by-block-table"))
 
     val content =
@@ -62,23 +53,23 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
           <ul class="unstyled">
             <li>
               <strong>Storage Level:</strong>
-              {rddInfo.storageLevel.description}
+              {rddStorageInfo.storageLevel}
             </li>
             <li>
               <strong>Cached Partitions:</strong>
-              {rddInfo.numCachedPartitions}
+              {rddStorageInfo.numCachedPartitions}
             </li>
             <li>
               <strong>Total Partitions:</strong>
-              {rddInfo.numPartitions}
+              {rddStorageInfo.numPartitions}
             </li>
             <li>
               <strong>Memory Size:</strong>
-              {Utils.bytesToString(rddInfo.memSize)}
+              {Utils.bytesToString(rddStorageInfo.memoryUsed)}
             </li>
             <li>
               <strong>Disk Size:</strong>
-              {Utils.bytesToString(rddInfo.diskSize)}
+              {Utils.bytesToString(rddStorageInfo.diskUsed)}
             </li>
           </ul>
         </div>
@@ -86,19 +77,19 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
 
       <div class="row-fluid">
         <div class="span12">
-          <h4> Data Distribution on {workers.size} Executors </h4>
+          <h4> Data Distribution on {rddStorageInfo.dataDistribution.size} Executors </h4>
           {workerTable}
         </div>
       </div>
 
       <div class="row-fluid">
         <div class="span12">
-          <h4> {blocks.size} Partitions </h4>
+          <h4> {rddStorageInfo.partitions.size} Partitions </h4>
           {blockTable}
         </div>
       </div>;
 
-    UIUtils.headerSparkPage("RDD Storage Info for " + rddInfo.name, content, parent)
+    UIUtils.headerSparkPage("RDD Storage Info for " + rddStorageInfo.name, content, parent)
   }
 
   /** Header fields for the worker table */
@@ -116,34 +107,32 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") {
     "Executors")
 
   /** Render an HTML row representing a worker */
-  private def workerRow(worker: (Int, StorageStatus)): Seq[Node] = {
-    val (rddId, status) = worker
+  private def workerRow(worker: RDDDataDistribution): Seq[Node] = {
     <tr>
-      <td>{status.blockManagerId.host + ":" + status.blockManagerId.port}</td>
+      <td>{worker.address}</td>
       <td>
-        {Utils.bytesToString(status.memUsedByRdd(rddId))}
-        ({Utils.bytesToString(status.memRemaining)} Remaining)
+        {Utils.bytesToString(worker.memoryUsed)}
+        ({Utils.bytesToString(worker.memoryRemaining)} Remaining)
       </td>
-      <td>{Utils.bytesToString(status.diskUsedByRdd(rddId))}</td>
+      <td>{Utils.bytesToString(worker.diskUsed)}</td>
     </tr>
   }
 
   /** Render an HTML row representing a block */
-  private def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = {
-    val (id, block, locations) = row
+  private def blockRow(row: RDDPartitionInfo): Seq[Node] = {
     <tr>
-      <td>{id}</td>
+      <td>{row.blockName}</td>
       <td>
-        {block.storageLevel.description}
+        {row.storageLevel}
       </td>
-      <td sorttable_customkey={block.memSize.toString}>
-        {Utils.bytesToString(block.memSize)}
+      <td sorttable_customkey={row.memoryUsed.toString}>
+        {Utils.bytesToString(row.memoryUsed)}
       </td>
-      <td sorttable_customkey={block.diskSize.toString}>
-        {Utils.bytesToString(block.diskSize)}
+      <td sorttable_customkey={row.diskUsed.toString}>
+        {Utils.bytesToString(row.diskUsed)}
       </td>
       <td>
-        {locations.map(l => <span>{l}<br/></span>)}
+        {row.executors.map(l => <span>{l}<br/></span>)}
       </td>
     </tr>
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
index 59dc6b5..07db783 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
@@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest
 import scala.xml.Node
 
 import org.apache.spark.storage.RDDInfo
-import org.apache.spark.ui.{WebUIPage, UIUtils}
+import org.apache.spark.ui.{UIUtils, WebUIPage}
 import org.apache.spark.util.Utils
 
 /** Page showing list of RDD's currently stored in the cluster */

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
index 045bd78..0351749 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
@@ -35,6 +35,8 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag
 /**
  * :: DeveloperApi ::
  * A SparkListener that prepares information to be displayed on the BlockManagerUI.
+ *
+ * This class is thread-safe (unlike JobProgressListener)
  */
 @DeveloperApi
 class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener {
@@ -43,7 +45,9 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar
   def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList
 
   /** Filter RDD info to include only those with cached partitions */
-  def rddInfoList: Seq[RDDInfo] = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq
+  def rddInfoList: Seq[RDDInfo] = synchronized {
+    _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq
+  }
 
   /** Update the storage info of the RDDs whose blocks are among the given updated blocks */
   private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation
new file mode 100644
index 0000000..6101177
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation
@@ -0,0 +1,53 @@
+[ {
+  "id" : "local-1427397477963",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-03-26T19:17:57.184GMT",
+    "endTime" : "2015-03-26T19:20:02.949GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1426533911241",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "attemptId" : "2",
+    "startTime" : "2015-03-17T23:11:50.242GMT",
+    "endTime" : "2015-03-17T23:12:25.177GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  }, {
+    "attemptId" : "1",
+    "startTime" : "2015-03-16T19:25:10.242GMT",
+    "endTime" : "2015-03-16T19:25:45.177GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1425081759269",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-28T00:02:38.277GMT",
+    "endTime" : "2015-02-28T00:02:46.912GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1422981780767",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-03T16:42:59.720GMT",
+    "endTime" : "2015-02-03T16:43:08.731GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}, {
+  "id" : "local-1422981759269",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-03T16:42:38.277GMT",
+    "endTime" : "2015-02-03T16:42:46.912GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation
new file mode 100644
index 0000000..cb622e1
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/executors/json_expectation
@@ -0,0 +1,17 @@
+[ {
+  "id" : "<driver>",
+  "hostPort" : "localhost:57971",
+  "rddBlocks" : 8,
+  "memoryUsed" : 28000128,
+  "diskUsed" : 0,
+  "activeTasks" : 0,
+  "failedTasks" : 1,
+  "completedTasks" : 31,
+  "totalTasks" : 32,
+  "totalDuration" : 8820,
+  "totalInputBytes" : 28000288,
+  "totalShuffleRead" : 0,
+  "totalShuffleWrite" : 13180,
+  "maxMemory" : 278302556,
+  "executorLogs" : { }
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation
new file mode 100644
index 0000000..4a29072
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/0/json_expectation
@@ -0,0 +1,15 @@
+{
+  "jobId" : 0,
+  "name" : "count at <console>:15",
+  "stageIds" : [ 0 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation
new file mode 100644
index 0000000..cab4750
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs/json_expectation
@@ -0,0 +1,43 @@
+[ {
+  "jobId" : 2,
+  "name" : "count at <console>:17",
+  "stageIds" : [ 3 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+}, {
+  "jobId" : 1,
+  "name" : "count at <console>:20",
+  "stageIds" : [ 1, 2 ],
+  "status" : "FAILED",
+  "numTasks" : 16,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 15,
+  "numSkippedTasks" : 15,
+  "numFailedTasks" : 1,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 1
+}, {
+  "jobId" : 0,
+  "name" : "count at <console>:15",
+  "stageIds" : [ 0 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation
new file mode 100644
index 0000000..cab4750
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded&status=failed/json_expectation
@@ -0,0 +1,43 @@
+[ {
+  "jobId" : 2,
+  "name" : "count at <console>:17",
+  "stageIds" : [ 3 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+}, {
+  "jobId" : 1,
+  "name" : "count at <console>:20",
+  "stageIds" : [ 1, 2 ],
+  "status" : "FAILED",
+  "numTasks" : 16,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 15,
+  "numSkippedTasks" : 15,
+  "numFailedTasks" : 1,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 1
+}, {
+  "jobId" : 0,
+  "name" : "count at <console>:15",
+  "stageIds" : [ 0 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation
new file mode 100644
index 0000000..6fd25be
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/jobs?status=succeeded/json_expectation
@@ -0,0 +1,29 @@
+[ {
+  "jobId" : 2,
+  "name" : "count at <console>:17",
+  "stageIds" : [ 3 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+}, {
+  "jobId" : 0,
+  "name" : "count at <console>:15",
+  "stageIds" : [ 0 ],
+  "status" : "SUCCEEDED",
+  "numTasks" : 8,
+  "numActiveTasks" : 0,
+  "numCompletedTasks" : 8,
+  "numSkippedTasks" : 8,
+  "numFailedTasks" : 0,
+  "numActiveStages" : 0,
+  "numCompletedStages" : 1,
+  "numSkippedStages" : 0,
+  "numFailedStages" : 0
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation
new file mode 100644
index 0000000..07489ad
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/json_expectation
@@ -0,0 +1,10 @@
+{
+  "id" : "local-1422981780767",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2015-02-03T16:42:59.720GMT",
+    "endTime" : "2015-02-03T16:43:08.731GMT",
+    "sparkUser" : "irashid",
+    "completed" : true
+  } ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation
new file mode 100644
index 0000000..111cb81
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/0/json_expectation
@@ -0,0 +1,270 @@
+{
+  "status" : "COMPLETE",
+  "stageId" : 1,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 3476,
+  "inputBytes" : 28000128,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 13180,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "map at <console>:14",
+  "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIM
 ain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ],
+  "tasks" : {
+    "8" : {
+      "taskId" : 8,
+      "index" : 0,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.829GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 1,
+        "executorRunTime" : 435,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 94000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "11" : {
+      "taskId" : 11,
+      "index" : 3,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.830GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1647,
+          "writeTime" : 83000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "14" : {
+      "taskId" : 14,
+      "index" : 6,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.832GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 88000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "13" : {
+      "taskId" : 13,
+      "index" : 5,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.831GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 73000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "10" : {
+      "taskId" : 10,
+      "index" : 2,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.830GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 76000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "9" : {
+      "taskId" : 9,
+      "index" : 1,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.830GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 1,
+        "executorRunTime" : 436,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 98000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "12" : {
+      "taskId" : 12,
+      "index" : 4,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.831GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1645,
+          "writeTime" : 101000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "15" : {
+      "taskId" : 15,
+      "index" : 7,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.833GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 1,
+        "executorRunTime" : 435,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 79000,
+          "recordsWritten" : 0
+        }
+      }
+    }
+  },
+  "executorSummary" : {
+    "<driver>" : {
+      "taskTime" : 3624,
+      "failedTasks" : 0,
+      "succeededTasks" : 8,
+      "inputBytes" : 28000128,
+      "outputBytes" : 0,
+      "shuffleRead" : 0,
+      "shuffleWrite" : 13180,
+      "memoryBytesSpilled" : 0,
+      "diskBytesSpilled" : 0
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation
new file mode 100644
index 0000000..ef339f8
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/1/json_expectation
@@ -0,0 +1,270 @@
+[ {
+  "status" : "COMPLETE",
+  "stageId" : 1,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 3476,
+  "inputBytes" : 28000128,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 13180,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "map at <console>:14",
+  "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIM
 ain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ],
+  "tasks" : {
+    "8" : {
+      "taskId" : 8,
+      "index" : 0,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.829GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 1,
+        "executorRunTime" : 435,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 94000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "11" : {
+      "taskId" : 11,
+      "index" : 3,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.830GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1647,
+          "writeTime" : 83000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "14" : {
+      "taskId" : 14,
+      "index" : 6,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.832GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 88000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "13" : {
+      "taskId" : 13,
+      "index" : 5,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.831GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 2,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 73000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "10" : {
+      "taskId" : 10,
+      "index" : 2,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.830GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 76000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "9" : {
+      "taskId" : 9,
+      "index" : 1,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.830GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 1,
+        "executorRunTime" : 436,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 98000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "12" : {
+      "taskId" : 12,
+      "index" : 4,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.831GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 2,
+        "executorRunTime" : 434,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1645,
+          "writeTime" : 101000,
+          "recordsWritten" : 0
+        }
+      }
+    },
+    "15" : {
+      "taskId" : 15,
+      "index" : 7,
+      "attempt" : 0,
+      "launchTime" : "2015-02-03T16:43:05.833GMT",
+      "executorId" : "<driver>",
+      "host" : "localhost",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 1,
+        "executorRunTime" : 435,
+        "resultSize" : 1902,
+        "jvmGcTime" : 19,
+        "resultSerializationTime" : 1,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "inputMetrics" : {
+          "bytesRead" : 3500016,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 1648,
+          "writeTime" : 79000,
+          "recordsWritten" : 0
+        }
+      }
+    }
+  },
+  "executorSummary" : {
+    "<driver>" : {
+      "taskTime" : 3624,
+      "failedTasks" : 0,
+      "succeededTasks" : 8,
+      "inputBytes" : 28000128,
+      "outputBytes" : 0,
+      "shuffleRead" : 0,
+      "shuffleWrite" : 13180,
+      "memoryBytesSpilled" : 0,
+      "diskBytesSpilled" : 0
+    }
+  }
+} ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/d4973580/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation
----------------------------------------------------------------------
diff --git a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation
new file mode 100644
index 0000000..056fac7
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation
@@ -0,0 +1,89 @@
+[ {
+  "status" : "COMPLETE",
+  "stageId" : 3,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 162,
+  "inputBytes" : 160,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "count at <console>:17",
+  "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line19.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line19.$read$$iwC$$iwC.<init>(<console>:24)\n$line19.$read$$iwC.<init>(<console>:26)\n$line19.$read.<init>(<console>:28)\n$line19.$read$.<init>(<console>:32)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.<init>(<console>:7)\n$line19.$eval$.<clinit>(<console>)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.Spark
 IMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ]
+}, {
+  "status" : "COMPLETE",
+  "stageId" : 1,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 3476,
+  "inputBytes" : 28000128,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 13180,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "map at <console>:14",
+  "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIM
 ain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ]
+}, {
+  "status" : "COMPLETE",
+  "stageId" : 0,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 8,
+  "numFailedTasks" : 0,
+  "executorRunTime" : 4338,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "count at <console>:15",
+  "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.inte
 rpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ]
+}, {
+  "status" : "FAILED",
+  "stageId" : 2,
+  "attemptId" : 0,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 7,
+  "numFailedTasks" : 1,
+  "executorRunTime" : 278,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteRecords" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "name" : "count at <console>:20",
+  "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.Spark
 IMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)",
+  "schedulingPool" : "default",
+  "accumulatorUpdates" : [ ]
+} ]
\ No newline at end of file


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