From 9c0c12564812bfa258d9080bea8fc984eeace191 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 16 Mar 2015 14:43:03 -0500 Subject: [PATCH] add accumulableInfo --- .../status/api/v1/AllStagesResource.scala | 11 +++++++++ .../status/api/v1/JsonRootResource.scala | 3 ++- .../org/apache/spark/status/api/v1/api.scala | 18 +++++++++++++- .../applications/json_expectation | 7 ++++++ .../stages/1/json_expectation | 9 +++++++ .../stages/json_expectation | 12 ++++++---- .../stages?status=complete/json_expectation | 9 ++++--- .../stages?status=failed/json_expectation | 3 ++- .../json_expectation | 7 ++++++ .../json_expectation | 7 ++++++ .../local-1426533911241/APPLICATION_COMPLETE | 0 .../local-1426533911241/EVENT_LOG_1 | 24 +++++++++++++++++++ .../local-1426533911241/SPARK_VERSION_1.2.0 | 0 13 files changed, 100 insertions(+), 10 deletions(-) create mode 100755 core/src/test/resources/spark-events/local-1426533911241/APPLICATION_COMPLETE create mode 100755 core/src/test/resources/spark-events/local-1426533911241/EVENT_LOG_1 create mode 100755 core/src/test/resources/spark-events/local-1426533911241/SPARK_VERSION_1.2.0 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 index 2794345763506..77d79a7199b79 100644 --- 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 @@ -21,6 +21,7 @@ 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} import org.apache.spark.scheduler.StageInfo import org.apache.spark.status.api._ import org.apache.spark.ui.SparkUI @@ -87,6 +88,9 @@ object AllStagesResource { } else { None } + + val accumulableInfo = stageUiData.accumulables.values.map { convertAccumulableInfo }.toSeq + new StageData( status = status, stageId = stageInfo.stageId, @@ -107,6 +111,7 @@ object AllStagesResource { schedulingPool = stageUiData.schedulingPool, name = stageInfo.name, details = stageInfo.details, + accumulatorUpdates = accumulableInfo, tasks = taskData, executorSummary = executorSummary ) @@ -135,11 +140,17 @@ object AllStagesResource { 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 convertAccumulableInfo(acc: InternalAccumulableInfo): AccumulableInfo = { + new AccumulableInfo(acc.id, acc.name, acc.value) + } + + def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = { new TaskMetrics( executorDeserializeTime = internal.executorDeserializeTime, 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 index e8b502f983ec4..15382bb30823c 100644 --- 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 @@ -101,7 +101,8 @@ private[spark] trait UIRoot { */ def withSparkUI[T](appId: String)(f: SparkUI => T): T = { getSparkUI(appId) match { - case Some(ui) => f(ui) + case Some(ui) => + f(ui) case None => throw new NotFoundException("no such app: " + appId) } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 04c3a19eecdfe..8005c5f4c2907 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -135,7 +135,7 @@ class StageData( val details: String, val schedulingPool: String, - //TODO what to do about accumulables? + val accumulatorUpdates: Seq[AccumulableInfo], val tasks: Option[Map[Long, TaskData]], val executorSummary:Option[Map[String,ExecutorStageSummary]] ) @@ -149,6 +149,7 @@ class TaskData( val host: String, val taskLocality: String, val speculative: Boolean, + val accumulatorUpdates: Seq[AccumulableInfo], val errorMessage: Option[String] = None, val taskMetrics: Option[TaskMetrics] = None ) @@ -191,3 +192,18 @@ class ShuffleWriteMetrics( val writeTime: Long, val recordsWritten: Long ) + +class AccumulableInfo ( + val id: Long, + val name: String, + //no partial updates, since they aren't logged. We can add them later + val value: String) { + + override def equals(other: Any): Boolean = other match { + case acc: AccumulableInfo => + this.id == acc.id && this.name == acc.name && + this.value == acc.value + case _ => false + } +} + diff --git a/core/src/test/resources/HistoryServerExpectations/applications/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation index 2321416365501..fdbdaf42b5e1a 100644 --- a/core/src/test/resources/HistoryServerExpectations/applications/json_expectation +++ b/core/src/test/resources/HistoryServerExpectations/applications/json_expectation @@ -1,4 +1,11 @@ [ { + "id" : "local-1426533911241", + "name" : "Spark shell", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true +}, { "id" : "local-1425081759269", "name" : "Spark shell", "startTime" : "2015-02-28T00:02:38.277GMT", 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 index 5ef8f4d190851..0693f8fed1172 100644 --- 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 @@ -18,6 +18,7 @@ "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\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)", "schedulingPool" : "default", + "accumulatorUpdates" : [ ], "tasks" : { "8" : { "taskId" : 8, @@ -28,6 +29,7 @@ "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, + "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 1, "executorRunTime" : 435, @@ -56,6 +58,7 @@ "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, + "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, "executorRunTime" : 434, @@ -84,6 +87,7 @@ "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, + "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, "executorRunTime" : 434, @@ -112,6 +116,7 @@ "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, + "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, "executorRunTime" : 434, @@ -140,6 +145,7 @@ "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, + "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, "executorRunTime" : 434, @@ -168,6 +174,7 @@ "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, + "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 1, "executorRunTime" : 436, @@ -196,6 +203,7 @@ "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, + "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 2, "executorRunTime" : 434, @@ -224,6 +232,7 @@ "host" : "localhost", "taskLocality" : "PROCESS_LOCAL", "speculative" : false, + "accumulatorUpdates" : [ ], "taskMetrics" : { "executorDeserializeTime" : 1, "executorRunTime" : 435, 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 index af84fc6dfdf9b..2dae2a3bc61f1 100644 --- a/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation +++ b/core/src/test/resources/HistoryServerExpectations/applications/local-1422981780767/stages/json_expectation @@ -17,7 +17,8 @@ "diskBytesSpilled" : 0, "name" : "count at :17", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\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)", - "schedulingPool" : "default" + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] }, { "status" : "Complete", "stageId" : 1, @@ -37,7 +38,8 @@ "diskBytesSpilled" : 0, "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\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)", - "schedulingPool" : "default" + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] }, { "status" : "Complete", "stageId" : 0, @@ -57,7 +59,8 @@ "diskBytesSpilled" : 0, "name" : "count at :15", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\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)", - "schedulingPool" : "default" + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] }, { "status" : "Failed", "stageId" : 2, @@ -77,5 +80,6 @@ "diskBytesSpilled" : 0, "name" : "count at :20", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\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)", - "schedulingPool" : "default" + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] } ] \ No newline at end of file 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 index 15cd8bcf0f563..6a06a6010c518 100644 --- 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 @@ -17,7 +17,8 @@ "diskBytesSpilled" : 0, "name" : "count at :17", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\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)", - "schedulingPool" : "default" + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] }, { "status" : "Complete", "stageId" : 1, @@ -37,7 +38,8 @@ "diskBytesSpilled" : 0, "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\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)", - "schedulingPool" : "default" + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] }, { "status" : "Complete", "stageId" : 0, @@ -57,5 +59,6 @@ "diskBytesSpilled" : 0, "name" : "count at :15", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\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)", - "schedulingPool" : "default" + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] } ] \ No newline at end of file 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 index 9f2c1325373ce..76157cc1ee210 100644 --- 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 @@ -17,5 +17,6 @@ "diskBytesSpilled" : 0, "name" : "count at :20", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\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)", - "schedulingPool" : "default" + "schedulingPool" : "default", + "accumulatorUpdates" : [ ] } ] \ No newline at end of file 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 index 343da9b61763e..07e21261affbb 100644 --- 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 @@ -1,4 +1,11 @@ [ { + "id" : "local-1426533911241", + "name" : "Spark shell", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true +}, { "id" : "local-1425081759269", "name" : "Spark shell", "startTime" : "2015-02-28T00:02:38.277GMT", diff --git a/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation b/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation index 2321416365501..fdbdaf42b5e1a 100644 --- a/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation +++ b/core/src/test/resources/HistoryServerExpectations/applications?status=completed/json_expectation @@ -1,4 +1,11 @@ [ { + "id" : "local-1426533911241", + "name" : "Spark shell", + "startTime" : "2015-03-16T19:25:10.242GMT", + "endTime" : "2015-03-16T19:25:45.177GMT", + "sparkUser" : "irashid", + "completed" : true +}, { "id" : "local-1425081759269", "name" : "Spark shell", "startTime" : "2015-02-28T00:02:38.277GMT", 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 0000000000000..e69de29bb2d1d 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 0000000000000..0fe9cdf15cd52 --- /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":"","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:/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","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 --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"} +{"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :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.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\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 :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.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\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":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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":"","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 :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.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\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)","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} 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 0000000000000..e69de29bb2d1d