From a22c45889d8fc0982caf4325eb729048537872bb Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 31 Jan 2017 13:31:55 -0800 Subject: [PATCH 1/6] [SPARK-20648][core] Port JobsTab and StageTab to the new UI backend. This change is a little larger because there's a whole lot of logic behind these pages, all really tied to internal types and listeners, and some of that logic had to be implemented in the new listener and the needed data exposed through the API types. - Added missing StageData and ExecutorStageSummary fields which are used by the UI. Some json golden files needed to be updated to account for new fields. - Save RDD graph data in the store. This tries to re-use existing types as much as possible, so that the code doesn't need to be re-written. So it's probably not very optimal. - Some old classes (e.g. JobProgressListener) still remain, since they're used in other parts of the code; they're not used by the UI anymore, though, and will be cleaned up in a separate change. - Save information about active pools in the store. This data is not really used in the SHS, but it's not a lot of data so it's still recorded when replaying applications. - Because the new store sorts things slightly differently from the previous code, some json golden files had some elements within them shuffled around. - The retention unit test in UISeleniumSuite was disabled because the code to throw away old stages / tasks hasn't been added yet. - The job description field in the API tries to follow the old behavior, which makes it be empty most of the time, even though there's information to fill it in. For stages, a new field was added to hold the description (which is basically the job description), so that the UI can be rendered in the old way. - A new stage status ("SKIPPED") was added to account for the fact that the API couldn't represent that state before. Without this, the stage would show up as "PENDING" in the UI, which is now based on API types. - The API used to expose "executorRunTime" as the value of the task's duration, which wasn't really correct (also because that value was easily available from the metrics object); this change fixes that by storing the correct duration, which also means a few expectation files needed to be updated to account for the new durations and sorting differences due to the changed values. - Added changes to implement SPARK-20713 and SPARK-21922 in the new code. --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../deploy/history/FsHistoryProvider.scala | 3 +- .../spark/status/AppStatusListener.scala | 121 +- .../apache/spark/status/AppStatusStore.scala | 114 +- .../org/apache/spark/status/LiveEntity.scala | 57 +- .../spark/status/api/v1/AllJobsResource.scala | 70 +- .../status/api/v1/AllStagesResource.scala | 290 +---- .../spark/status/api/v1/OneJobResource.scala | 15 +- .../status/api/v1/OneStageResource.scala | 112 +- .../org/apache/spark/status/api/v1/api.scala | 21 +- .../org/apache/spark/status/config.scala | 4 + .../org/apache/spark/status/storeTypes.scala | 40 + .../scala/org/apache/spark/ui/SparkUI.scala | 30 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 286 +++-- .../apache/spark/ui/jobs/AllStagesPage.scala | 191 +-- .../apache/spark/ui/jobs/ExecutorTable.scala | 159 +-- .../org/apache/spark/ui/jobs/JobPage.scala | 327 ++--- .../org/apache/spark/ui/jobs/JobsTab.scala | 45 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 59 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 34 +- .../org/apache/spark/ui/jobs/StagePage.scala | 1073 ++++++++--------- .../org/apache/spark/ui/jobs/StageTable.scala | 100 +- .../org/apache/spark/ui/jobs/StagesTab.scala | 31 +- .../spark/ui/scope/RDDOperationGraph.scala | 10 +- .../ui/scope/RDDOperationGraphListener.scala | 150 --- .../complete_stage_list_json_expectation.json | 21 +- .../failed_stage_list_json_expectation.json | 8 +- ...multi_attempt_app_json_1__expectation.json | 5 +- ...multi_attempt_app_json_2__expectation.json | 5 +- .../job_list_json_expectation.json | 15 +- .../one_job_json_expectation.json | 5 +- .../one_stage_attempt_json_expectation.json | 126 +- .../one_stage_json_expectation.json | 126 +- .../stage_list_json_expectation.json | 79 +- ...ist_with_accumulable_json_expectation.json | 7 +- .../stage_task_list_expectation.json | 60 +- ...multi_attempt_app_json_1__expectation.json | 24 +- ...multi_attempt_app_json_2__expectation.json | 24 +- ...k_list_w__offset___length_expectation.json | 150 ++- ...stage_task_list_w__sortBy_expectation.json | 190 +-- ...tBy_short_names___runtime_expectation.json | 190 +-- ...rtBy_short_names__runtime_expectation.json | 60 +- ...age_with_accumulable_json_expectation.json | 36 +- ...eded_failed_job_list_json_expectation.json | 15 +- .../succeeded_job_list_json_expectation.json | 10 +- .../deploy/history/HistoryServerSuite.scala | 13 +- .../spark/status/AppStatusListenerSuite.scala | 80 +- .../api/v1/AllStagesResourceSuite.scala | 62 - .../org/apache/spark/ui/StagePageSuite.scala | 64 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 60 +- .../RDDOperationGraphListenerSuite.scala | 226 ---- project/MimaExcludes.scala | 2 + 52 files changed, 2376 insertions(+), 2631 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala delete mode 100644 core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphListenerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e5aaaf6c155e..29b82c15bc93 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -426,7 +426,7 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the app status store and listener before SparkEnv is created so that it gets // all events. - _statusStore = AppStatusStore.createLiveStore(conf, listenerBus) + _statusStore = AppStatusStore.createLiveStore(conf, l => listenerBus.addToStatusQueue(l)) // Create the Spark execution environment (cache, map output tracker, etc) _env = createSparkEnv(_conf, isLocal, listenerBus) 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 f16dddea9f78..1df31ef32642 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 @@ -316,7 +316,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } val listener = if (needReplay) { - val _listener = new AppStatusListener(kvstore, conf, false) + val _listener = new AppStatusListener(kvstore, conf, false, + lastUpdateTime = Some(attempt.info.lastUpdated.getTime())) replayBus.addListener(_listener) Some(_listener) } else { diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 0469c871362c..007234e81c6f 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -28,16 +28,21 @@ import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1 import org.apache.spark.storage._ import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.scope._ import org.apache.spark.util.kvstore.KVStore /** * A Spark listener that writes application information to a data store. The types written to the * store are defined in the `storeTypes.scala` file and are based on the public REST API. + * + * @param lastUpdateTime When replaying logs, the log's last update time, so that the duration of + * unfinished tasks can be more accurately calculated (see SPARK-21922). */ private[spark] class AppStatusListener( kvstore: KVStore, conf: SparkConf, - live: Boolean) extends SparkListener with Logging { + live: Boolean, + lastUpdateTime: Option[Long] = None) extends SparkListener with Logging { import config._ @@ -50,6 +55,8 @@ private[spark] class AppStatusListener( // operations that we can live without when rapidly processing incoming task events. private val liveUpdatePeriodNs = if (live) conf.get(LIVE_ENTITY_UPDATE_PERIOD) else -1L + private val maxGraphRootNodes = conf.get(MAX_RETAINED_ROOT_NODES) + // Keep track of live entities, so that task metrics can be efficiently updated (without // causing too many writes to the underlying store, and other expensive operations). private val liveStages = new HashMap[(Int, Int), LiveStage]() @@ -57,6 +64,7 @@ private[spark] class AppStatusListener( private val liveExecutors = new HashMap[String, LiveExecutor]() private val liveTasks = new HashMap[Long, LiveTask]() private val liveRDDs = new HashMap[Int, LiveRDD]() + private val pools = new HashMap[String, SchedulerPool]() override def onOtherEvent(event: SparkListenerEvent): Unit = event match { case SparkListenerLogStart(version) => sparkVersion = version @@ -201,16 +209,15 @@ private[spark] class AppStatusListener( missingStages.map(_.numTasks).sum } - val lastStageInfo = event.stageInfos.lastOption + val lastStageInfo = event.stageInfos.sortBy(_.stageId).lastOption val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val jobGroup = Option(event.properties) .flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) } val job = new LiveJob( event.jobId, lastStageName, - Some(new Date(event.time)), + if (event.time > 0) Some(new Date(event.time)) else None, event.stageIds, jobGroup, numTasks) @@ -225,17 +232,51 @@ private[spark] class AppStatusListener( stage.jobIds += event.jobId liveUpdate(stage, now) } + + // Create the graph data for all the job's stages. + event.stageInfos.foreach { stage => + val graph = RDDOperationGraph.makeOperationGraph(stage, maxGraphRootNodes) + val uigraph = new RDDOperationGraphWrapper( + stage.stageId, + graph.edges, + graph.outgoingEdges, + graph.incomingEdges, + newRDDOperationCluster(graph.rootCluster)) + kvstore.write(uigraph) + } + } + + private def newRDDOperationCluster(cluster: RDDOperationCluster): RDDOperationClusterWrapper = { + new RDDOperationClusterWrapper( + cluster.id, + cluster.name, + cluster.childNodes, + cluster.childClusters.map(newRDDOperationCluster)) } override def onJobEnd(event: SparkListenerJobEnd): Unit = { liveJobs.remove(event.jobId).foreach { job => + val now = System.nanoTime() + + // Check if there are any pending stages that match this job; mark those as skipped. + job.stageIds.foreach { sid => + val pending = liveStages.filter { case ((id, _), _) => id == sid } + pending.foreach { case (key, stage) => + stage.status = v1.StageStatus.SKIPPED + job.skippedStages += stage.info.stageId + job.skippedTasks += stage.info.numTasks + liveStages.remove(key) + update(stage, now) + } + } + job.status = event.jobResult match { case JobSucceeded => JobExecutionStatus.SUCCEEDED case JobFailed(_) => JobExecutionStatus.FAILED } - job.completionTime = Some(new Date(event.time)) - update(job, System.nanoTime()) + job.completionTime = if (event.time > 0) Some(new Date(event.time)) else None + update(job, now) } } @@ -253,12 +294,24 @@ private[spark] class AppStatusListener( .toSeq stage.jobIds = stage.jobs.map(_.jobId).toSet + stage.schedulingPool = Option(event.properties).flatMap { p => + Option(p.getProperty("spark.scheduler.pool")) + }.getOrElse(SparkUI.DEFAULT_POOL_NAME) + + stage.description = Option(event.properties).flatMap { p => + Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) + } + stage.jobs.foreach { job => job.completedStages = job.completedStages - event.stageInfo.stageId job.activeStages += 1 liveUpdate(job, now) } + val pool = pools.getOrElseUpdate(stage.schedulingPool, new SchedulerPool(stage.schedulingPool)) + pool.stageIds = pool.stageIds + event.stageInfo.stageId + update(pool, now) + event.stageInfo.rddInfos.foreach { info => if (info.storageLevel.isValid) { liveUpdate(liveRDDs.getOrElseUpdate(info.id, new LiveRDD(info)), now) @@ -270,7 +323,7 @@ private[spark] class AppStatusListener( override def onTaskStart(event: SparkListenerTaskStart): Unit = { val now = System.nanoTime() - val task = new LiveTask(event.taskInfo, event.stageId, event.stageAttemptId) + val task = new LiveTask(event.taskInfo, event.stageId, event.stageAttemptId, lastUpdateTime) liveTasks.put(event.taskInfo.taskId, task) liveUpdate(task, now) @@ -309,6 +362,8 @@ private[spark] class AppStatusListener( val now = System.nanoTime() val metricsDelta = liveTasks.remove(event.taskInfo.taskId).map { task => + task.info = event.taskInfo + val errorMessage = event.reason match { case Success => None @@ -328,11 +383,15 @@ private[spark] class AppStatusListener( delta }.orNull - val (completedDelta, failedDelta) = event.reason match { + val (completedDelta, failedDelta, killedDelta) = event.reason match { case Success => - (1, 0) + (1, 0, 0) + case _: TaskKilled => + (0, 0, 1) + case _: TaskCommitDenied => + (0, 0, 1) case _ => - (0, 1) + (0, 1, 0) } liveStages.get((event.stageId, event.stageAttemptId)).foreach { stage => @@ -341,13 +400,29 @@ private[spark] class AppStatusListener( } stage.activeTasks -= 1 stage.completedTasks += completedDelta + if (completedDelta > 0) { + stage.completedIndices.add(event.taskInfo.index) + } stage.failedTasks += failedDelta + stage.killedTasks += killedDelta + if (killedDelta > 0) { + stage.killedSummary = killedTasksSummary(event.reason, stage.killedSummary) + } maybeUpdate(stage, now) + // Store both stage ID and task index in a single long variable for tracking at job level. + val taskIndex = (event.stageId.toLong << Integer.SIZE) | event.taskInfo.index stage.jobs.foreach { job => job.activeTasks -= 1 job.completedTasks += completedDelta + if (completedDelta > 0) { + job.completedIndices.add(taskIndex) + } job.failedTasks += failedDelta + job.killedTasks += killedDelta + if (killedDelta > 0) { + job.killedSummary = killedTasksSummary(event.reason, job.killedSummary) + } maybeUpdate(job, now) } @@ -355,6 +430,7 @@ private[spark] class AppStatusListener( esummary.taskTime += event.taskInfo.duration esummary.succeededTasks += completedDelta esummary.failedTasks += failedDelta + esummary.killedTasks += killedDelta if (metricsDelta != null) { esummary.metrics.update(metricsDelta) } @@ -413,6 +489,11 @@ private[spark] class AppStatusListener( liveUpdate(job, now) } + pools.get(stage.schedulingPool).foreach { pool => + pool.stageIds = pool.stageIds - event.stageInfo.stageId + update(pool, now) + } + stage.executorSummaries.values.foreach(update(_, now)) update(stage, now) } @@ -472,11 +553,15 @@ private[spark] class AppStatusListener( /** Flush all live entities' data to the underlying store. */ def flush(): Unit = { val now = System.nanoTime() - liveStages.values.foreach(update(_, now)) + liveStages.values.foreach { stage => + update(stage, now) + stage.executorSummaries.values.foreach(update(_, now)) + } liveJobs.values.foreach(update(_, now)) liveExecutors.values.foreach(update(_, now)) liveTasks.values.foreach(update(_, now)) liveRDDs.values.foreach(update(_, now)) + pools.values.foreach(update(_, now)) } private def updateRDDBlock(event: SparkListenerBlockUpdated, block: RDDBlockId): Unit = { @@ -583,6 +668,20 @@ private[spark] class AppStatusListener( stage } + private def killedTasksSummary( + reason: TaskEndReason, + oldSummary: Map[String, Int]): Map[String, Int] = { + reason match { + case k: TaskKilled => + oldSummary.updated(k.reason, oldSummary.getOrElse(k.reason, 0) + 1) + case denied: TaskCommitDenied => + val reason = denied.toErrorString + oldSummary.updated(reason, oldSummary.getOrElse(reason, 0) + 1) + case _ => + oldSummary + } + } + private def update(entity: LiveEntity, now: Long): Unit = { entity.write(kvstore, now) } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 334407829f9f..cdf790730ff1 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -23,8 +23,9 @@ import java.util.{Arrays, List => JList} import scala.collection.JavaConverters._ import org.apache.spark.{JobExecutionStatus, SparkConf} -import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.scheduler.SparkListener import org.apache.spark.status.api.v1 +import org.apache.spark.ui.scope._ import org.apache.spark.util.{Distribution, Utils} import org.apache.spark.util.kvstore.{InMemoryStore, KVStore} @@ -43,8 +44,8 @@ private[spark] class AppStatusStore(store: KVStore) { } def jobsList(statuses: JList[JobExecutionStatus]): Seq[v1.JobData] = { - val it = store.view(classOf[JobDataWrapper]).asScala.map(_.info) - if (!statuses.isEmpty()) { + val it = store.view(classOf[JobDataWrapper]).reverse().asScala.map(_.info) + if (statuses != null && !statuses.isEmpty()) { it.filter { job => statuses.contains(job.status) }.toSeq } else { it.toSeq @@ -75,21 +76,35 @@ private[spark] class AppStatusStore(store: KVStore) { } def stageList(statuses: JList[v1.StageStatus]): Seq[v1.StageData] = { - val it = store.view(classOf[StageDataWrapper]).asScala.map(_.info) - if (!statuses.isEmpty) { + val it = store.view(classOf[StageDataWrapper]).reverse().asScala.map(_.info) + if (statuses != null && !statuses.isEmpty()) { it.filter { s => statuses.contains(s.status) }.toSeq } else { it.toSeq } } - def stageData(stageId: Int): Seq[v1.StageData] = { + def stageData(stageId: Int, details: Boolean = false): Seq[v1.StageData] = { store.view(classOf[StageDataWrapper]).index("stageId").first(stageId).last(stageId) - .asScala.map(_.info).toSeq + .asScala.map { s => + if (details) stageWithDetails(s.info) else s.info + }.toSeq } - def stageAttempt(stageId: Int, stageAttemptId: Int): v1.StageData = { - store.read(classOf[StageDataWrapper], Array(stageId, stageAttemptId)).info + def lastStageAttempt(stageId: Int): v1.StageData = { + val it = store.view(classOf[StageDataWrapper]).index("stageId").reverse().first(stageId) + .closeableIterator() + try { + it.next().info + } finally { + it.close() + } + } + + def stageAttempt(stageId: Int, stageAttemptId: Int, details: Boolean = false): v1.StageData = { + val stageKey = Array(stageId, stageAttemptId) + val stage = store.read(classOf[StageDataWrapper], stageKey).info + if (details) stageWithDetails(stage) else stage } def taskSummary( @@ -189,6 +204,12 @@ private[spark] class AppStatusStore(store: KVStore) { ) } + def taskList(stageId: Int, stageAttemptId: Int, maxTasks: Int): Seq[v1.TaskData] = { + val stageKey = Array(stageId, stageAttemptId) + store.view(classOf[TaskDataWrapper]).index("stage").first(stageKey).last(stageKey).reverse() + .max(maxTasks).asScala.map(_.info).toSeq.reverse + } + def taskList( stageId: Int, stageAttemptId: Int, @@ -213,10 +234,79 @@ private[spark] class AppStatusStore(store: KVStore) { store.view(classOf[RDDStorageInfoWrapper]).asScala.map(_.info).toSeq } + private def stageWithDetails(stage: v1.StageData): v1.StageData = { + val tasks = taskList(stage.stageId, stage.attemptId, Int.MaxValue) + .map { t => (t.taskId, t) } + .toMap + + val stageKey = Array(stage.stageId, stage.attemptId) + val execs = store.view(classOf[ExecutorStageSummaryWrapper]).index("stage").first(stageKey) + .last(stageKey).closeableIterator().asScala + .map { exec => (exec.executorId -> exec.info) } + .toMap + + new v1.StageData( + stage.status, + stage.stageId, + stage.attemptId, + stage.numTasks, + stage.numActiveTasks, + stage.numCompleteTasks, + stage.numFailedTasks, + stage.numKilledTasks, + stage.numCompletedIndices, + stage.executorRunTime, + stage.executorCpuTime, + stage.submissionTime, + stage.firstTaskLaunchedTime, + stage.completionTime, + stage.failureReason, + stage.inputBytes, + stage.inputRecords, + stage.outputBytes, + stage.outputRecords, + stage.shuffleReadBytes, + stage.shuffleReadRecords, + stage.shuffleWriteBytes, + stage.shuffleWriteRecords, + stage.memoryBytesSpilled, + stage.diskBytesSpilled, + stage.name, + stage.description, + stage.details, + stage.schedulingPool, + stage.rddIds, + stage.accumulatorUpdates, + Some(tasks), + Some(execs), + stage.killedTasksSummary) + } + def rdd(rddId: Int): v1.RDDStorageInfo = { store.read(classOf[RDDStorageInfoWrapper], rddId).info } + def operationGraphForStage(stageId: Int): RDDOperationGraph = { + store.read(classOf[RDDOperationGraphWrapper], stageId).toRDDOperationGraph() + } + + def operationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = { + val job = store.read(classOf[JobDataWrapper], jobId) + val stages = job.info.stageIds + + stages.map { id => + val g = store.read(classOf[RDDOperationGraphWrapper], id).toRDDOperationGraph() + if (job.skippedStages.contains(id) && !g.rootCluster.name.contains("skipped")) { + g.rootCluster.setName(g.rootCluster.name + " (skipped)") + } + g + } + } + + def pool(name: String): PoolData = { + store.read(classOf[PoolData], name) + } + def close(): Unit = { store.close() } @@ -231,12 +321,12 @@ private[spark] object AppStatusStore { * Create an in-memory store for a live application. * * @param conf Configuration. - * @param bus Where to attach the listener to populate the store. + * @param addListenerFn Function to register a listener with a bus. */ - def createLiveStore(conf: SparkConf, bus: LiveListenerBus): AppStatusStore = { + def createLiveStore(conf: SparkConf, addListenerFn: SparkListener => Unit): AppStatusStore = { val store = new InMemoryStore() val stateStore = new AppStatusStore(store) - bus.addToStatusQueue(new AppStatusListener(store, conf, true)) + addListenerFn(new AppStatusListener(store, conf, true)) stateStore } diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 8c48020e246b..93c5e91a482c 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -28,6 +28,7 @@ import org.apache.spark.status.api.v1 import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.SparkUI import org.apache.spark.util.AccumulatorContext +import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.kvstore.KVStore /** @@ -64,6 +65,12 @@ private class LiveJob( var completedTasks = 0 var failedTasks = 0 + // Holds both the stage ID and the task index, packed into a single long value. + val completedIndices = new OpenHashSet[Long]() + + var killedTasks = 0 + var killedSummary: Map[String, Int] = Map() + var skippedTasks = 0 var skippedStages = Set[Int]() @@ -89,19 +96,23 @@ private class LiveJob( completedTasks, skippedTasks, failedTasks, + killedTasks, + completedIndices.size, activeStages, completedStages.size, skippedStages.size, - failedStages) + failedStages, + killedSummary) new JobDataWrapper(info, skippedStages) } } private class LiveTask( - info: TaskInfo, + var info: TaskInfo, stageId: Int, - stageAttemptId: Int) extends LiveEntity { + stageAttemptId: Int, + lastUpdateTime: Option[Long]) extends LiveEntity { import LiveEntityHelpers._ @@ -126,6 +137,7 @@ private class LiveTask( metrics.resultSerializationTime, metrics.memoryBytesSpilled, metrics.diskBytesSpilled, + metrics.peakExecutionMemory, new v1.InputMetrics( metrics.inputMetrics.bytesRead, metrics.inputMetrics.recordsRead), @@ -186,6 +198,7 @@ private class LiveTask( 0L, 0L, 0L, metrics.memoryBytesSpilled - old.memoryBytesSpilled, metrics.diskBytesSpilled - old.diskBytesSpilled, + 0L, inputDelta, outputDelta, shuffleReadDelta, @@ -193,12 +206,19 @@ private class LiveTask( } override protected def doUpdate(): Any = { + val duration = if (info.finished) { + info.duration + } else { + info.timeRunning(lastUpdateTime.getOrElse(System.currentTimeMillis())) + } + val task = new v1.TaskData( info.taskId, info.index, info.attemptNumber, new Date(info.launchTime), - if (info.finished) Some(info.duration) else None, + if (info.gettingResult) Some(new Date(info.gettingResultTime)) else None, + Some(duration), info.executorId, info.host, info.status, @@ -340,10 +360,15 @@ private class LiveExecutorStageSummary( taskTime, failedTasks, succeededTasks, + killedTasks, metrics.inputBytes, + metrics.inputRecords, metrics.outputBytes, + metrics.outputRecords, metrics.shuffleReadBytes, + metrics.shuffleReadRecords, metrics.shuffleWriteBytes, + metrics.shuffleWriteRecords, metrics.memoryBytesSpilled, metrics.diskBytesSpilled) new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info) @@ -361,11 +386,16 @@ private class LiveStage extends LiveEntity { var info: StageInfo = null var status = v1.StageStatus.PENDING + var description: Option[String] = None var schedulingPool: String = SparkUI.DEFAULT_POOL_NAME var activeTasks = 0 var completedTasks = 0 var failedTasks = 0 + val completedIndices = new OpenHashSet[Int]() + + var killedTasks = 0 + var killedSummary: Map[String, Int] = Map() var firstLaunchTime = Long.MaxValue @@ -384,15 +414,19 @@ private class LiveStage extends LiveEntity { info.stageId, info.attemptId, + info.numTasks, activeTasks, completedTasks, failedTasks, + killedTasks, + completedIndices.size, metrics.executorRunTime, metrics.executorCpuTime, info.submissionTime.map(new Date(_)), if (firstLaunchTime < Long.MaxValue) Some(new Date(firstLaunchTime)) else None, info.completionTime.map(new Date(_)), + info.failureReason, metrics.inputBytes, metrics.inputRecords, @@ -406,12 +440,15 @@ private class LiveStage extends LiveEntity { metrics.diskBytesSpilled, info.name, + description, info.details, schedulingPool, + info.rddInfos.map(_.id), newAccumulatorInfos(info.accumulables.values), None, - None) + None, + killedSummary) new StageDataWrapper(update, jobIds) } @@ -513,6 +550,16 @@ private class LiveRDD(info: RDDInfo) extends LiveEntity { } +private class SchedulerPool(name: String) extends LiveEntity { + + var stageIds = Set[Int]() + + override protected def doUpdate(): Any = { + new PoolData(name, stageIds) + } + +} + private object LiveEntityHelpers { def newAccumulatorInfos(accums: Iterable[AccumulableInfo]): Seq[v1.AccumulableInfo] = { 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 index d0d9ef1165e8..b4fa3e633f6c 100644 --- 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 @@ -22,7 +22,6 @@ 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)) @@ -30,74 +29,7 @@ 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} + ui.store.jobsList(statuses) } } - -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 = - if (job.stageIds.isEmpty) { - None - } else { - 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.numSkippedTasks, - numFailedTasks = job.numFailedTasks, - numActiveStages = job.numActiveStages, - numCompletedStages = job.completedStageIndices.size, - numSkippedStages = job.numSkippedStages, - numFailedStages = job.numFailedStages - ) - } - } -} 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 5f69949c618f..e1c91cb527a5 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 @@ -16,304 +16,18 @@ */ package org.apache.spark.status.api.v1 -import java.util.{Arrays, Date, List => JList} +import java.util.{List => JList} import javax.ws.rs.{GET, Produces, QueryParam} import javax.ws.rs.core.MediaType -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.ui.jobs.UIData.{InputMetricsUIData => InternalInputMetrics, OutputMetricsUIData => InternalOutputMetrics, ShuffleReadMetricsUIData => InternalShuffleReadMetrics, ShuffleWriteMetricsUIData => InternalShuffleWriteMetrics, TaskMetricsUIData => InternalTaskMetrics} -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 { - stageUiData.lastUpdateTime = ui.lastUpdateTime - AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, includeDetails = false) - } + ui.store.stageList(statuses) } -} - -private[v1] object AllStagesResource { - def stageUiToStageData( - status: StageStatus, - stageInfo: StageInfo, - stageUiData: StageUIData, - includeDetails: Boolean): StageData = { - - val taskLaunchTimes = stageUiData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0) - - val firstTaskLaunchedTime: Option[Date] = - if (taskLaunchTimes.nonEmpty) { - Some(new Date(taskLaunchTimes.min)) - } else { - None - } - - val taskData = if (includeDetails) { - Some(stageUiData.taskData.map { case (k, v) => - k -> convertTaskData(v, stageUiData.lastUpdateTime) }.toMap) - } 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 - ) - }.toMap) - } 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, - executorCpuTime = stageUiData.executorCpuTime, - submissionTime = stageInfo.submissionTime.map(new Date(_)), - firstTaskLaunchedTime, - completionTime = stageInfo.completionTime.map(new Date(_)), - 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, lastUpdateTime: Option[Long]): TaskData = { - new TaskData( - taskId = uiData.taskInfo.taskId, - index = uiData.taskInfo.index, - attempt = uiData.taskInfo.attemptNumber, - launchTime = new Date(uiData.taskInfo.launchTime), - duration = uiData.taskDuration(lastUpdateTime), - executorId = uiData.taskInfo.executorId, - host = uiData.taskInfo.host, - status = uiData.taskInfo.status, - taskLocality = uiData.taskInfo.taskLocality.toString(), - speculative = uiData.taskInfo.speculative, - accumulatorUpdates = uiData.taskInfo.accumulables.map { convertAccumulableInfo }, - errorMessage = uiData.errorMessage, - taskMetrics = uiData.metrics.map { convertUiTaskMetrics } - ) - } - - def taskMetricDistributions( - allTaskData: Iterable[TaskUIData], - quantiles: Array[Double]): TaskMetricDistributions = { - - val rawMetrics = allTaskData.flatMap{_.metrics}.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: InputMetricDistributions = - new MetricHelper[InternalInputMetrics, InputMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): InternalInputMetrics = raw.inputMetrics - - def build: InputMetricDistributions = new InputMetricDistributions( - bytesRead = submetricQuantiles(_.bytesRead), - recordsRead = submetricQuantiles(_.recordsRead) - ) - }.build - - val outputMetrics: OutputMetricDistributions = - new MetricHelper[InternalOutputMetrics, OutputMetricDistributions](rawMetrics, quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): InternalOutputMetrics = raw.outputMetrics - - def build: OutputMetricDistributions = new OutputMetricDistributions( - bytesWritten = submetricQuantiles(_.bytesWritten), - recordsWritten = submetricQuantiles(_.recordsWritten) - ) - }.build - - val shuffleReadMetrics: ShuffleReadMetricDistributions = - new MetricHelper[InternalShuffleReadMetrics, ShuffleReadMetricDistributions](rawMetrics, - quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): InternalShuffleReadMetrics = - raw.shuffleReadMetrics - - def build: ShuffleReadMetricDistributions = new ShuffleReadMetricDistributions( - readBytes = submetricQuantiles(_.totalBytesRead), - readRecords = submetricQuantiles(_.recordsRead), - remoteBytesRead = submetricQuantiles(_.remoteBytesRead), - remoteBytesReadToDisk = submetricQuantiles(_.remoteBytesReadToDisk), - remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched), - localBlocksFetched = submetricQuantiles(_.localBlocksFetched), - totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched), - fetchWaitTime = submetricQuantiles(_.fetchWaitTime) - ) - }.build - - val shuffleWriteMetrics: ShuffleWriteMetricDistributions = - new MetricHelper[InternalShuffleWriteMetrics, ShuffleWriteMetricDistributions](rawMetrics, - quantiles) { - def getSubmetrics(raw: InternalTaskMetrics): InternalShuffleWriteMetrics = - raw.shuffleWriteMetrics - def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( - writeBytes = submetricQuantiles(_.bytesWritten), - writeRecords = submetricQuantiles(_.recordsWritten), - writeTime = submetricQuantiles(_.writeTime) - ) - }.build - - new TaskMetricDistributions( - quantiles = quantiles, - executorDeserializeTime = metricQuantiles(_.executorDeserializeTime), - executorDeserializeCpuTime = metricQuantiles(_.executorDeserializeCpuTime), - executorRunTime = metricQuantiles(_.executorRunTime), - executorCpuTime = metricQuantiles(_.executorCpuTime), - 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.orNull, acc.update.map(_.toString), acc.value.map(_.toString).orNull) - } - - def convertUiTaskMetrics(internal: InternalTaskMetrics): TaskMetrics = { - new TaskMetrics( - executorDeserializeTime = internal.executorDeserializeTime, - executorDeserializeCpuTime = internal.executorDeserializeCpuTime, - executorRunTime = internal.executorRunTime, - executorCpuTime = internal.executorCpuTime, - resultSize = internal.resultSize, - jvmGcTime = internal.jvmGCTime, - resultSerializationTime = internal.resultSerializationTime, - memoryBytesSpilled = internal.memoryBytesSpilled, - diskBytesSpilled = internal.diskBytesSpilled, - inputMetrics = convertInputMetrics(internal.inputMetrics), - outputMetrics = convertOutputMetrics(internal.outputMetrics), - shuffleReadMetrics = convertShuffleReadMetrics(internal.shuffleReadMetrics), - shuffleWriteMetrics = convertShuffleWriteMetrics(internal.shuffleWriteMetrics) - ) - } - - 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, - remoteBytesReadToDisk = internal.remoteBytesReadToDisk, - localBytesRead = internal.localBytesRead, - recordsRead = internal.recordsRead - ) - } - - def convertShuffleWriteMetrics(internal: InternalShuffleWriteMetrics): ShuffleWriteMetrics = { - new ShuffleWriteMetrics( - bytesWritten = internal.bytesWritten, - writeTime = internal.writeTime, - recordsWritten = internal.recordsWritten - ) - } -} - -/** - * Helper for getting distributions from nested metric types. - */ -private[v1] abstract class MetricHelper[I, O]( - rawMetrics: Seq[InternalTaskMetrics], - quantiles: Array[Double]) { - - def getSubmetrics(raw: InternalTaskMetrics): I - - def build: O - - val data: Seq[I] = rawMetrics.map(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) - } } 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 index 653150385c73..3ee884e084c1 100644 --- 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 @@ -16,25 +16,22 @@ */ package org.apache.spark.status.api.v1 +import java.util.NoSuchElementException import javax.ws.rs.{GET, PathParam, 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.flatMap(_._2).find { jobInfo => jobInfo.jobId == jobId} - jobOpt.map { job => - AllJobsResource.convertJobData(job, ui.jobProgressListener, false) - }.getOrElse { - throw new NotFoundException("unknown job: " + jobId) + try { + ui.store.job(jobId) + } catch { + case _: NoSuchElementException => + throw new NotFoundException("unknown job: " + jobId) } } 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 index f15073bccced..20dd73e91661 100644 --- 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 @@ -24,7 +24,6 @@ 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)) @@ -32,13 +31,14 @@ private[v1] class OneStageResource(ui: SparkUI) { @GET @Path("") - def stageData(@PathParam("stageId") stageId: Int): Seq[StageData] = { - withStage(stageId) { stageAttempts => - stageAttempts.map { stage => - stage.ui.lastUpdateTime = ui.lastUpdateTime - AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, - includeDetails = true) - } + def stageData( + @PathParam("stageId") stageId: Int, + @QueryParam("details") @DefaultValue("true") details: Boolean): Seq[StageData] = { + val ret = ui.store.stageData(stageId, details = details) + if (ret.nonEmpty) { + ret + } else { + throw new NotFoundException(s"unknown stage: $stageId") } } @@ -46,11 +46,13 @@ private[v1] class OneStageResource(ui: SparkUI) { @Path("/{stageAttemptId: \\d+}") def oneAttemptData( @PathParam("stageId") stageId: Int, - @PathParam("stageAttemptId") stageAttemptId: Int): StageData = { - withStageAttempt(stageId, stageAttemptId) { stage => - stage.ui.lastUpdateTime = ui.lastUpdateTime - AllStagesResource.stageUiToStageData(stage.status, stage.info, stage.ui, - includeDetails = true) + @PathParam("stageAttemptId") stageAttemptId: Int, + @QueryParam("details") @DefaultValue("true") details: Boolean): StageData = { + try { + ui.store.stageAttempt(stageId, stageAttemptId, details = details) + } catch { + case _: NoSuchElementException => + throw new NotFoundException(s"unknown attempt $stageAttemptId for stage $stageId.") } } @@ -61,17 +63,16 @@ private[v1] class OneStageResource(ui: SparkUI) { @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) - } + 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) } + + ui.store.taskSummary(stageId, stageAttemptId, quantiles) } @GET @@ -82,72 +83,7 @@ private[v1] class OneStageResource(ui: SparkUI) { @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(_, ui.lastUpdateTime)}.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) - } + ui.store.taskList(stageId, stageAttemptId, offset, length, sortBy) } - 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) - } } 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 b338b1f3fd07..14280099f642 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 @@ -58,10 +58,15 @@ class ExecutorStageSummary private[spark]( val taskTime : Long, val failedTasks : Int, val succeededTasks : Int, + val killedTasks : Int, val inputBytes : Long, + val inputRecords : Long, val outputBytes : Long, + val outputRecords : Long, val shuffleRead : Long, + val shuffleReadRecords : Long, val shuffleWrite : Long, + val shuffleWriteRecords : Long, val memoryBytesSpilled : Long, val diskBytesSpilled : Long) @@ -111,10 +116,13 @@ class JobData private[spark]( val numCompletedTasks: Int, val numSkippedTasks: Int, val numFailedTasks: Int, + val numKilledTasks: Int, + val numCompletedIndices: Int, val numActiveStages: Int, val numCompletedStages: Int, val numSkippedStages: Int, - val numFailedStages: Int) + val numFailedStages: Int, + val killedTasksSummary: Map[String, Int]) class RDDStorageInfo private[spark]( val id: Int, @@ -152,15 +160,19 @@ class StageData private[spark]( val status: StageStatus, val stageId: Int, val attemptId: Int, + val numTasks: Int, val numActiveTasks: Int, val numCompleteTasks: Int, val numFailedTasks: Int, + val numKilledTasks: Int, + val numCompletedIndices: Int, val executorRunTime: Long, val executorCpuTime: Long, val submissionTime: Option[Date], val firstTaskLaunchedTime: Option[Date], val completionTime: Option[Date], + val failureReason: Option[String], val inputBytes: Long, val inputRecords: Long, @@ -174,18 +186,22 @@ class StageData private[spark]( val diskBytesSpilled: Long, val name: String, + val description: Option[String], val details: String, val schedulingPool: String, + val rddIds: Seq[Int], val accumulatorUpdates: Seq[AccumulableInfo], val tasks: Option[Map[Long, TaskData]], - val executorSummary: Option[Map[String, ExecutorStageSummary]]) + val executorSummary: Option[Map[String, ExecutorStageSummary]], + val killedTasksSummary: Map[String, Int]) class TaskData private[spark]( val taskId: Long, val index: Int, val attempt: Int, val launchTime: Date, + val resultFetchStart: Option[Date], @JsonDeserialize(contentAs = classOf[JLong]) val duration: Option[Long], val executorId: String, @@ -207,6 +223,7 @@ class TaskMetrics private[spark]( val resultSerializationTime: Long, val memoryBytesSpilled: Long, val diskBytesSpilled: Long, + val peakExecutionMemory: Long, val inputMetrics: InputMetrics, val outputMetrics: OutputMetrics, val shuffleReadMetrics: ShuffleReadMetrics, diff --git a/core/src/main/scala/org/apache/spark/status/config.scala b/core/src/main/scala/org/apache/spark/status/config.scala index 49144fc883e6..7af9dff977a8 100644 --- a/core/src/main/scala/org/apache/spark/status/config.scala +++ b/core/src/main/scala/org/apache/spark/status/config.scala @@ -27,4 +27,8 @@ private[spark] object config { .timeConf(TimeUnit.NANOSECONDS) .createWithDefaultString("100ms") + val MAX_RETAINED_ROOT_NODES = ConfigBuilder("spark.ui.dagGraph.retainedRootRDDs") + .intConf + .createWithDefault(Int.MaxValue) + } diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index 23e9a360ddc0..b0002ba151da 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore import org.apache.spark.status.KVUtils._ import org.apache.spark.status.api.v1._ +import org.apache.spark.ui.scope._ import org.apache.spark.util.kvstore.KVIndex private[spark] case class AppStatusStoreMetadata(version: Long) @@ -106,6 +107,11 @@ private[spark] class TaskDataWrapper( Array(stageId: JInteger, stageAttemptId: JInteger, _runtime: JLong) } + @JsonIgnore @KVIndex("startTime") + def startTime: Array[AnyRef] = { + Array(stageId: JInteger, stageAttemptId: JInteger, info.launchTime.getTime(): JLong) + } + } private[spark] class RDDStorageInfoWrapper(val info: RDDStorageInfo) { @@ -131,3 +137,37 @@ private[spark] class ExecutorStageSummaryWrapper( private[this] val stage: Array[Int] = Array(stageId, stageAttemptId) } + +private[spark] class RDDOperationClusterWrapper( + val id: String, + val name: String, + val childNodes: Seq[RDDOperationNode], + val childClusters: Seq[RDDOperationClusterWrapper]) { + + def toRDDOperationCluster(): RDDOperationCluster = { + val cluster = new RDDOperationCluster(id, name) + childNodes.foreach(cluster.attachChildNode) + childClusters.foreach { child => + cluster.attachChildCluster(child.toRDDOperationCluster()) + } + cluster + } + +} + +private[spark] class RDDOperationGraphWrapper( + @KVIndexParam val stageId: Int, + val edges: Seq[RDDOperationEdge], + val outgoingEdges: Seq[RDDOperationEdge], + val incomingEdges: Seq[RDDOperationEdge], + val rootCluster: RDDOperationClusterWrapper) { + + def toRDDOperationGraph(): RDDOperationGraph = { + new RDDOperationGraph(edges, outgoingEdges, incomingEdges, rootCluster.toRDDOperationCluster()) + } + +} + +private[spark] class PoolData( + @KVIndexParam val name: String, + val stageIds: Set[Int]) 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 79d40b6a90c3..0f1c425e2197 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,11 +17,11 @@ package org.apache.spark.ui -import java.util.{Date, ServiceLoader} +import java.util.{Date, List => JList, ServiceLoader} import scala.collection.JavaConverters._ -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{JobExecutionStatus, SecurityManager, SparkConf, SparkContext} import org.apache.spark.internal.Logging import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore @@ -30,8 +30,7 @@ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentTab import org.apache.spark.ui.exec.ExecutorsTab -import org.apache.spark.ui.jobs.{JobProgressListener, JobsTab, StagesTab} -import org.apache.spark.ui.scope.RDDOperationGraphListener +import org.apache.spark.ui.jobs.{JobsTab, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} import org.apache.spark.util.Utils @@ -44,12 +43,9 @@ private[spark] class SparkUI private ( val conf: SparkConf, securityManager: SecurityManager, val storageStatusListener: StorageStatusListener, - val jobProgressListener: JobProgressListener, val storageListener: StorageListener, - val operationGraphListener: RDDOperationGraphListener, var appName: String, val basePath: String, - val lastUpdateTime: Option[Long] = None, val startTime: Long, val appSparkVersion: String) extends WebUI(securityManager, securityManager.getSSLOptions("ui"), SparkUI.getUIPort(conf), @@ -64,8 +60,8 @@ private[spark] class SparkUI private ( private var streamingJobProgressListener: Option[SparkListener] = None /** Initialize all components of the server. */ - def initialize() { - val jobsTab = new JobsTab(this) + def initialize(): Unit = { + val jobsTab = new JobsTab(this, store) attachTab(jobsTab) val stagesTab = new StagesTab(this, store) attachTab(stagesTab) @@ -75,6 +71,7 @@ private[spark] class SparkUI private ( attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs/", basePath = basePath)) attachHandler(ApiRootResource.getServletHandler(this)) + // These should be POST only, but, the YARN AM proxy won't proxy POSTs attachHandler(createRedirectHandler( "/jobs/job/kill", "/jobs/", jobsTab.handleKillRequest, httpMethods = Set("GET", "POST"))) @@ -82,6 +79,7 @@ private[spark] class SparkUI private ( "/stages/stage/kill", "/stages/", stagesTab.handleKillRequest, httpMethods = Set("GET", "POST"))) } + initialize() def getSparkUser: String = { @@ -178,26 +176,16 @@ private[spark] object SparkUI { appName: String, basePath: String, startTime: Long, - lastUpdateTime: Option[Long] = None, appSparkVersion: String = org.apache.spark.SPARK_VERSION): SparkUI = { - val jobProgressListener = sc.map(_.jobProgressListener).getOrElse { - val listener = new JobProgressListener(conf) - addListenerFn(listener) - listener - } - val storageStatusListener = new StorageStatusListener(conf) val storageListener = new StorageListener(storageStatusListener) - val operationGraphListener = new RDDOperationGraphListener(conf) addListenerFn(storageStatusListener) addListenerFn(storageListener) - addListenerFn(operationGraphListener) - new SparkUI(store, sc, conf, securityManager, storageStatusListener, jobProgressListener, - storageListener, operationGraphListener, appName, basePath, lastUpdateTime, startTime, - appSparkVersion) + new SparkUI(store, sc, conf, securityManager, storageStatusListener, storageListener, + appName, basePath, startTime, appSparkVersion) } } 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 a647a1173a8c..617ba05bc414 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 @@ -23,19 +23,21 @@ import javax.servlet.http.HttpServletRequest import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} +import scala.util.Try import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler._ -import org.apache.spark.status.api.v1.ExecutorSummary +import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.api.v1 import org.apache.spark.ui._ import org.apache.spark.ui.jobs.UIData.{JobUIData, StageUIData} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished jobs */ -private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { +private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends WebUIPage("") { private val JOBS_LEGEND =
Removed
.toString.filter(_ != '\n') - private def getLastStageNameAndDescription(job: JobUIData): (String, String) = { - val lastStageInfo = Option(job.stageIds) - .filter(_.nonEmpty) - .flatMap { ids => parent.jobProgresslistener.stageIdToInfo.get(ids.max)} - val lastStageData = lastStageInfo.flatMap { s => - parent.jobProgresslistener.stageIdToData.get((s.stageId, s.attemptId)) - } - val name = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val description = lastStageData.flatMap(_.description).getOrElse("") - (name, description) - } - - private def makeJobEvent(jobUIDatas: Seq[JobUIData]): Seq[String] = { - jobUIDatas.filter { jobUIData => - jobUIData.status != JobExecutionStatus.UNKNOWN && jobUIData.submissionTime.isDefined - }.map { jobUIData => - val jobId = jobUIData.jobId - val status = jobUIData.status - val (jobName, jobDescription) = getLastStageNameAndDescription(jobUIData) + private def makeJobEvent(jobs: Seq[v1.JobData]): Seq[String] = { + jobs.filter { job => + job.status != JobExecutionStatus.UNKNOWN && job.submissionTime.isDefined + }.map { job => + val jobId = job.jobId + val status = job.status val displayJobDescription = - if (jobDescription.isEmpty) { - jobName + if (job.description.isEmpty) { + job.name } else { - UIUtils.makeDescription(jobDescription, "", plainText = true).text + UIUtils.makeDescription(job.description.get, "", plainText = true).text } - val submissionTime = jobUIData.submissionTime.get - val completionTimeOpt = jobUIData.completionTime - val completionTime = completionTimeOpt.getOrElse(System.currentTimeMillis()) + val submissionTime = job.submissionTime.get.getTime() + val completionTime = job.completionTime.map(_.getTime()).getOrElse(System.currentTimeMillis()) val classNameByStatus = status match { case JobExecutionStatus.SUCCEEDED => "succeeded" case JobExecutionStatus.FAILED => "failed" @@ -124,7 +112,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } } - private def makeExecutorEvent(executors: Seq[ExecutorSummary]): + private def makeExecutorEvent(executors: Seq[v1.ExecutorSummary]): Seq[String] = { val events = ListBuffer[String]() executors.foreach { e => @@ -169,8 +157,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } private def makeTimeline( - jobs: Seq[JobUIData], - executors: Seq[ExecutorSummary], + jobs: Seq[v1.JobData], + executors: Seq[v1.ExecutorSummary], startTime: Long): Seq[Node] = { val jobEventJsonAsStrSeq = makeJobEvent(jobs) @@ -217,7 +205,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { request: HttpServletRequest, tableHeaderId: String, jobTag: String, - jobs: Seq[JobUIData], + jobs: Seq[v1.JobData], killEnabled: Boolean): Seq[Node] = { // stripXSS is called to remove suspicious characters used in XSS attacks val allParameters = request.getParameterMap.asScala.toMap.mapValues(_.map(UIUtils.stripXSS)) @@ -258,14 +246,13 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { try { new JobPagedTable( + store, jobs, tableHeaderId, jobTag, UIUtils.prependBaseUri(parent.basePath), "jobs", // subPath parameterOtherTable, - parent.jobProgresslistener.stageIdToInfo, - parent.jobProgresslistener.stageIdToData, killEnabled, currentTime, jobIdTitle, @@ -285,106 +272,120 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } def render(request: HttpServletRequest): Seq[Node] = { - val listener = parent.jobProgresslistener - listener.synchronized { - val startTime = listener.startTime - val endTime = listener.endTime - val activeJobs = listener.activeJobs.values.toSeq - val completedJobs = listener.completedJobs.reverse - val failedJobs = listener.failedJobs.reverse - - val activeJobsTable = - jobsTable(request, "active", "activeJob", activeJobs, killEnabled = parent.killEnabled) - val completedJobsTable = - jobsTable(request, "completed", "completedJob", completedJobs, killEnabled = false) - val failedJobsTable = - jobsTable(request, "failed", "failedJob", failedJobs, killEnabled = false) - - val shouldShowActiveJobs = activeJobs.nonEmpty - 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 appInfo = store.applicationInfo() + val startTime = appInfo.attempts.head.startTime.getTime() + val endTime = appInfo.attempts.head.endTime.getTime() + + val activeJobs = new ListBuffer[v1.JobData]() + val _completedJobs = new ListBuffer[v1.JobData]() + val _failedJobs = new ListBuffer[v1.JobData]() + + store.jobsList(null).foreach { job => + job.status match { + case JobExecutionStatus.SUCCEEDED => + _completedJobs += job + case JobExecutionStatus.FAILED => + _failedJobs += job + case _ => + activeJobs += job } + } - val summary: NodeSeq = -
- +
- var content = summary - content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs, - parent.parent.store.executorList(false), startTime) + var content = summary + content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs, + store.executorList(false), startTime) - if (shouldShowActiveJobs) { - content ++=

Active Jobs ({activeJobs.size})

++ - activeJobsTable - } - if (shouldShowCompletedJobs) { - content ++=

Completed Jobs ({completedJobNumStr})

++ - completedJobsTable - } - if (shouldShowFailedJobs) { - content ++=

Failed Jobs ({failedJobs.size})

++ - failedJobsTable - } + if (shouldShowActiveJobs) { + content ++=

Active Jobs ({activeJobs.size})

++ + activeJobsTable + } + if (shouldShowCompletedJobs) { + content ++=

Completed Jobs ({completedJobNumStr})

++ + completedJobsTable + } + if (shouldShowFailedJobs) { + content ++=

Failed Jobs ({failedJobs.size})

++ + failedJobsTable + } - val helpText = """A job is triggered by an action, like count() or saveAsTextFile().""" + - " Click on a job to see information about the stages of tasks inside it." + val helpText = """A job is triggered by an action, like count() or saveAsTextFile().""" + + " Click on a job to see information about the stages of tasks inside it." - UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) - } + UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) } + } private[ui] class JobTableRowData( - val jobData: JobUIData, + val jobData: v1.JobData, val lastStageName: String, val lastStageDescription: String, val duration: Long, @@ -395,9 +396,8 @@ private[ui] class JobTableRowData( val detailUrl: String) private[ui] class JobDataSource( - jobs: Seq[JobUIData], - stageIdToInfo: HashMap[Int, StageInfo], - stageIdToData: HashMap[(Int, Int), StageUIData], + store: AppStatusStore, + jobs: Seq[v1.JobData], basePath: String, currentTime: Long, pageSize: Int, @@ -418,40 +418,28 @@ private[ui] class JobDataSource( r } - private def getLastStageNameAndDescription(job: JobUIData): (String, String) = { - val lastStageInfo = Option(job.stageIds) - .filter(_.nonEmpty) - .flatMap { ids => stageIdToInfo.get(ids.max)} - val lastStageData = lastStageInfo.flatMap { s => - stageIdToData.get((s.stageId, s.attemptId)) - } - val name = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") - val description = lastStageData.flatMap(_.description).getOrElse("") - (name, description) - } - - private def jobRow(jobData: JobUIData): JobTableRowData = { - val (lastStageName, lastStageDescription) = getLastStageNameAndDescription(jobData) + private def jobRow(jobData: v1.JobData): JobTableRowData = { val duration: Option[Long] = { jobData.submissionTime.map { start => - val end = jobData.completionTime.getOrElse(System.currentTimeMillis()) - end - start + val end = jobData.completionTime.map(_.getTime()).getOrElse(System.currentTimeMillis()) + end - start.getTime() } } val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val submissionTime = jobData.submissionTime val formattedSubmissionTime = submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") - val jobDescription = UIUtils.makeDescription(lastStageDescription, basePath, plainText = false) + val jobDescription = UIUtils.makeDescription(jobData.description.getOrElse(""), + basePath, plainText = false) val detailUrl = "%s/jobs/job?id=%s".format(basePath, jobData.jobId) - new JobTableRowData ( + new JobTableRowData( jobData, - lastStageName, - lastStageDescription, + jobData.name, + jobData.description.getOrElse(jobData.name), duration.getOrElse(-1), formattedDuration, - submissionTime.getOrElse(-1), + submissionTime.map(_.getTime()).getOrElse(-1L), formattedSubmissionTime, jobDescription, detailUrl @@ -480,14 +468,13 @@ private[ui] class JobDataSource( } private[ui] class JobPagedTable( - data: Seq[JobUIData], + store: AppStatusStore, + data: Seq[v1.JobData], tableHeaderId: String, jobTag: String, basePath: String, subPath: String, parameterOtherTable: Iterable[String], - stageIdToInfo: HashMap[Int, StageInfo], - stageIdToData: HashMap[(Int, Int), StageUIData], killEnabled: Boolean, currentTime: Long, jobIdTitle: String, @@ -510,9 +497,8 @@ private[ui] class JobPagedTable( override def pageNumberFormField: String = jobTag + ".page" override val dataSource = new JobDataSource( + store, data, - stageIdToInfo, - stageIdToData, basePath, currentTime, pageSize, @@ -624,15 +610,15 @@ private[ui] class JobPagedTable( {jobTableRow.formattedDuration} - {job.completedStageIndices.size}/{job.stageIds.size - job.numSkippedStages} + {job.numCompletedStages}/{job.stageIds.size - job.numSkippedStages} {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} {UIUtils.makeProgressBar(started = job.numActiveTasks, - completed = job.completedIndices.size, + completed = job.numCompletedIndices, failed = job.numFailedTasks, skipped = job.numSkippedTasks, - reasonToNumKilled = job.reasonToNumKilled, total = job.numTasks - job.numSkippedTasks)} + reasonToNumKilled = job.killedTasksSummary, total = job.numTasks - job.numSkippedTasks)} } 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 dc5b03c5269a..78381754bd1b 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 @@ -22,120 +22,125 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable +import org.apache.spark.status.PoolData +import org.apache.spark.status.api.v1._ import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { private val sc = parent.sc - private val listener = parent.progressListener private def isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { - listener.synchronized { - val activeStages = listener.activeStages.values.toSeq - val pendingStages = listener.pendingStages.values.toSeq - val completedStages = listener.completedStages.reverse - val numCompletedStages = listener.numCompletedStages - val failedStages = listener.failedStages.reverse - val numFailedStages = listener.numFailedStages - val subPath = "stages" + val allStages = parent.store.stageList(null) - val activeStagesTable = - new StageTableBase(request, activeStages, "active", "activeStage", parent.basePath, subPath, - parent.progressListener, parent.isFairScheduler, - killEnabled = parent.killEnabled, isFailedStage = false) - val pendingStagesTable = - new StageTableBase(request, pendingStages, "pending", "pendingStage", parent.basePath, - subPath, parent.progressListener, parent.isFairScheduler, - killEnabled = false, isFailedStage = false) - val completedStagesTable = - new StageTableBase(request, completedStages, "completed", "completedStage", parent.basePath, - subPath, parent.progressListener, parent.isFairScheduler, - killEnabled = false, isFailedStage = false) - val failedStagesTable = - new StageTableBase(request, failedStages, "failed", "failedStage", parent.basePath, subPath, - parent.progressListener, parent.isFairScheduler, - killEnabled = false, isFailedStage = true) + val activeStages = allStages.filter(_.status == StageStatus.ACTIVE) + val pendingStages = allStages.filter(_.status == StageStatus.PENDING) + val completedStages = allStages.filter(_.status == StageStatus.COMPLETE) + val failedStages = allStages.filter(_.status == StageStatus.FAILED).reverse - // For now, pool information is only accessible in live UIs - val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) - val poolTable = new PoolTable(pools, parent) + val numCompletedStages = completedStages.size + val numFailedStages = failedStages.size + val subPath = "stages" - val shouldShowActiveStages = activeStages.nonEmpty - val shouldShowPendingStages = pendingStages.nonEmpty - val shouldShowCompletedStages = completedStages.nonEmpty - val shouldShowFailedStages = failedStages.nonEmpty + val activeStagesTable = + new StageTableBase(parent.store, request, activeStages, "active", "activeStage", + parent.basePath, subPath, parent.isFairScheduler, parent.killEnabled, false) + val pendingStagesTable = + new StageTableBase(parent.store, request, pendingStages, "pending", "pendingStage", + parent.basePath, subPath, parent.isFairScheduler, false, false) + val completedStagesTable = + new StageTableBase(parent.store, request, completedStages, "completed", "completedStage", + parent.basePath, subPath, parent.isFairScheduler, false, false) + val failedStagesTable = + new StageTableBase(parent.store, request, failedStages, "failed", "failedStage", + parent.basePath, subPath, parent.isFairScheduler, false, true) - val completedStageNumStr = if (numCompletedStages == completedStages.size) { - s"$numCompletedStages" - } else { - s"$numCompletedStages, only showing ${completedStages.size}" + // For now, pool information is only accessible in live UIs + val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]).map { pool => + val uiPool = try { + parent.store.pool(pool.name) + } catch { + case _: NoSuchElementException => + new PoolData(pool.name, Set()) } + pool -> uiPool + }.toMap + val poolTable = new PoolTable(pools, parent) - val summary: NodeSeq = -
- +
+ + var content = summary ++ + { + if (sc.isDefined && isFairScheduler) { +

Fair Scheduler Pools ({pools.size})

++ poolTable.toNodeSeq + } else { + Seq.empty[Node] } - if (shouldShowActiveStages) { - content ++=

Active Stages ({activeStages.size})

++ - activeStagesTable.toNodeSeq } - if (shouldShowPendingStages) { - content ++=

Pending Stages ({pendingStages.size})

++ - pendingStagesTable.toNodeSeq - } - if (shouldShowCompletedStages) { - content ++=

Completed Stages ({completedStageNumStr})

++ - completedStagesTable.toNodeSeq - } - if (shouldShowFailedStages) { - content ++=

Failed Stages ({numFailedStages})

++ - failedStagesTable.toNodeSeq - } - UIUtils.headerSparkPage("Stages for All Jobs", content, parent) + if (shouldShowActiveStages) { + content ++=

Active Stages ({activeStages.size})

++ + activeStagesTable.toNodeSeq + } + if (shouldShowPendingStages) { + content ++=

Pending Stages ({pendingStages.size})

++ + pendingStagesTable.toNodeSeq } + if (shouldShowCompletedStages) { + content ++=

Completed Stages ({completedStageNumStr})

++ + completedStagesTable.toNodeSeq + } + if (shouldShowFailedStages) { + content ++=

Failed Stages ({numFailedStages})

++ + failedStagesTable.toNodeSeq + } + UIUtils.headerSparkPage("Stages for All Jobs", content, parent) } } - diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 07a41d195a19..12b3b99ed1a9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -17,44 +17,22 @@ package org.apache.spark.ui.jobs +import java.util.NoSuchElementException + import scala.collection.mutable import scala.xml.{Node, Unparsed} import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.api.v1.StageData import org.apache.spark.ui.{ToolTips, UIUtils} -import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils /** Stage summary grouped by executors. */ -private[ui] class ExecutorTable( - stageId: Int, - stageAttemptId: Int, - parent: StagesTab, - store: AppStatusStore) { - private val listener = parent.progressListener - - def toNodeSeq: Seq[Node] = { - listener.synchronized { - executorTable() - } - } +private[ui] class ExecutorTable(stage: StageData, store: AppStatusStore) { - /** Special table which merges two header cells. */ - private def executorTable[T](): Seq[Node] = { - val stageData = listener.stageIdToData.get((stageId, stageAttemptId)) - var hasInput = false - var hasOutput = false - var hasShuffleWrite = false - var hasShuffleRead = false - var hasBytesSpilled = false - stageData.foreach { data => - hasInput = data.hasInput - hasOutput = data.hasOutput - hasShuffleRead = data.hasShuffleRead - hasShuffleWrite = data.hasShuffleWrite - hasBytesSpilled = data.hasBytesSpilled - } + import ApiHelper._ + def toNodeSeq: Seq[Node] = { @@ -64,29 +42,29 @@ private[ui] class ExecutorTable( - {if (hasInput) { + {if (hasInput(stage)) { }} - {if (hasOutput) { + {if (hasOutput(stage)) { }} - {if (hasShuffleRead) { + {if (hasShuffleRead(stage)) { }} - {if (hasShuffleWrite) { + {if (hasShuffleWrite(stage)) { }} - {if (hasBytesSpilled) { + {if (hasBytesSpilled(stage)) { }} @@ -97,7 +75,7 @@ private[ui] class ExecutorTable( - {createExecutorTable()} + {createExecutorTable(stage)}
Executor IDFailed Tasks Killed Tasks Succeeded Tasks Input Size / Records Output Size / Records Shuffle Read Size / Records Shuffle Write Size / Records Shuffle Spill (Memory) Shuffle Spill (Disk)
} - private def createExecutorTable() : Seq[Node] = { - // Make an executor-id -> address map - val executorIdToAddress = mutable.HashMap[String, String]() - listener.blockManagerIds.foreach { blockManagerId => - val address = blockManagerId.hostPort - val executorId = blockManagerId.executorId - executorIdToAddress.put(executorId, address) - } - - listener.stageIdToData.get((stageId, stageAttemptId)) match { - case Some(stageData: StageUIData) => - stageData.executorSummary.toSeq.sortBy(_._1).map { case (k, v) => - - -
{k}
-
- { - store.executorSummary(k).map(_.executorLogs).getOrElse(Map.empty).map { - case (logName, logUrl) =>
{logName}
- } - } -
- - {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {UIUtils.formatDuration(v.taskTime)} - {v.failedTasks + v.succeededTasks + v.reasonToNumKilled.values.sum} - {v.failedTasks} - {v.reasonToNumKilled.values.sum} - {v.succeededTasks} - {if (stageData.hasInput) { - - {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"} - - }} - {if (stageData.hasOutput) { - - {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"} - - }} - {if (stageData.hasShuffleRead) { - - {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"} - - }} - {if (stageData.hasShuffleWrite) { - - {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"} - - }} - {if (stageData.hasBytesSpilled) { - - {Utils.bytesToString(v.memoryBytesSpilled)} - - - {Utils.bytesToString(v.diskBytesSpilled)} - - }} - {v.isBlacklisted} - - } - case None => - Seq.empty[Node] + private def createExecutorTable(stage: StageData) : Seq[Node] = { + stage.executorSummary.getOrElse(Map.empty).toSeq.sortBy(_._1).map { case (k, v) => + val executor = store.executorSummary(k) + + +
{k}
+
+ { + executor.map(_.executorLogs).getOrElse(Map.empty).map { + case (logName, logUrl) =>
{logName}
+ } + } +
+ + {executor.map { e => e.hostPort }.getOrElse("CANNOT FIND ADDRESS")} + {UIUtils.formatDuration(v.taskTime)} + {v.failedTasks + v.succeededTasks + v.killedTasks} + {v.failedTasks} + {v.killedTasks} + {v.succeededTasks} + {if (hasInput(stage)) { + + {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"} + + }} + {if (hasOutput(stage)) { + + {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"} + + }} + {if (hasShuffleRead(stage)) { + + {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"} + + }} + {if (hasShuffleWrite(stage)) { + + {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"} + + }} + {if (hasBytesSpilled(stage)) { + + {Utils.bytesToString(v.memoryBytesSpilled)} + + + {Utils.bytesToString(v.diskBytesSpilled)} + + }} + {executor.map(_.isBlacklisted).getOrElse(false)} + } } + } 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 7ed01646f362..15c51f7efc4c 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 @@ -21,17 +21,19 @@ import java.util.{Date, Locale} import javax.servlet.http.HttpServletRequest import scala.collection.mutable.{Buffer, ListBuffer} +import scala.util.Try import scala.xml.{Node, NodeSeq, Unparsed, Utility} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler._ -import org.apache.spark.status.api.v1.ExecutorSummary -import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} +import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.api.v1 +import org.apache.spark.ui._ /** Page showing statistics and stage list for a given job */ -private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { +private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIPage("job") { private val STAGES_LEGEND =
@@ -56,14 +58,15 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { Removed
.toString.filter(_ != '\n') - private def makeStageEvent(stageInfos: Seq[StageInfo]): Seq[String] = { + private def makeStageEvent(stageInfos: Seq[v1.StageData]): Seq[String] = { stageInfos.map { stage => val stageId = stage.stageId val attemptId = stage.attemptId val name = stage.name - val status = stage.getStatusString - val submissionTime = stage.submissionTime.get - val completionTime = stage.completionTime.getOrElse(System.currentTimeMillis()) + val status = stage.status.toString + val submissionTime = stage.submissionTime.get.getTime() + val completionTime = stage.completionTime.map(_.getTime()) + .getOrElse(System.currentTimeMillis()) // The timeline library treats contents as HTML, so we have to escape them. We need to add // extra layers of escaping in order to embed this in a Javascript string literal. @@ -79,10 +82,10 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { | 'data-placement="top" data-html="true"' + | 'data-title="${jsEscapedName} (Stage ${stageId}.${attemptId})
' + | 'Status: ${status.toUpperCase(Locale.ROOT)}
' + - | 'Submitted: ${UIUtils.formatDate(new Date(submissionTime))}' + + | 'Submitted: ${UIUtils.formatDate(submissionTime)}' + | '${ if (status != "running") { - s"""
Completed: ${UIUtils.formatDate(new Date(completionTime))}""" + s"""
Completed: ${UIUtils.formatDate(completionTime)}""" } else { "" } @@ -93,7 +96,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { } } - def makeExecutorEvent(executors: Seq[ExecutorSummary]): Seq[String] = { + def makeExecutorEvent(executors: Seq[v1.ExecutorSummary]): Seq[String] = { val events = ListBuffer[String]() executors.foreach { e => val addedEvent = @@ -137,8 +140,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { } private def makeTimeline( - stages: Seq[StageInfo], - executors: Seq[ExecutorSummary], + stages: Seq[v1.StageData], + executors: Seq[v1.ExecutorSummary], appStartTime: Long): Seq[Node] = { val stageEventJsonAsStrSeq = makeStageEvent(stages) @@ -182,173 +185,183 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { } def render(request: HttpServletRequest): Seq[Node] = { - val listener = parent.jobProgresslistener + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - listener.synchronized { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) - require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - - val jobId = parameterId.toInt - val jobDataOption = listener.jobIdToData.get(jobId) - if (jobDataOption.isEmpty) { - val content = -
-

No information to display for job {jobId}

-
- return UIUtils.headerSparkPage( - s"Details for Job $jobId", content, parent) - } - val jobData = jobDataOption.get - val isComplete = jobData.status != JobExecutionStatus.RUNNING - val stages = jobData.stageIds.map { stageId => - // This could be empty if the JobProgressListener hasn't received information about the - // stage or if the stage information has been garbage collected - listener.stageIdToInfo.getOrElse(stageId, - new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown")) + val jobId = parameterId.toInt + val jobDataOption = Try(store.job(jobId)).toOption + if (jobDataOption.isEmpty) { + val content = +
+

No information to display for job {jobId}

+
+ return UIUtils.headerSparkPage( + s"Details for Job $jobId", content, parent) + } + val jobData = jobDataOption.get + val isComplete = jobData.status != JobExecutionStatus.RUNNING + val stages = jobData.stageIds.map { stageId => + // This could be empty if the listener hasn't received information about the + // stage or if the stage information has been garbage collected + store.stageData(stageId).lastOption.getOrElse { + new v1.StageData( + v1.StageStatus.PENDING, + stageId, + 0, 0, 0, 0, 0, 0, 0, + 0L, 0L, None, None, None, None, + 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, + "Unknown", + None, + "Unknown", + null, + Nil, + Nil, + None, + None, + Map()) } + } - val activeStages = Buffer[StageInfo]() - val completedStages = Buffer[StageInfo]() - // If the job is completed, then any pending stages are displayed as "skipped": - val pendingOrSkippedStages = Buffer[StageInfo]() - val failedStages = Buffer[StageInfo]() - for (stage <- stages) { - if (stage.submissionTime.isEmpty) { - pendingOrSkippedStages += stage - } else if (stage.completionTime.isDefined) { - if (stage.failureReason.isDefined) { - failedStages += stage - } else { - completedStages += stage - } + val activeStages = Buffer[v1.StageData]() + val completedStages = Buffer[v1.StageData]() + // If the job is completed, then any pending stages are displayed as "skipped": + val pendingOrSkippedStages = Buffer[v1.StageData]() + val failedStages = Buffer[v1.StageData]() + for (stage <- stages) { + if (stage.submissionTime.isEmpty) { + pendingOrSkippedStages += stage + } else if (stage.completionTime.isDefined) { + if (stage.status == v1.StageStatus.FAILED) { + failedStages += stage } else { - activeStages += stage + completedStages += stage } + } else { + activeStages += stage } + } - val basePath = "jobs/job" + val basePath = "jobs/job" - val pendingOrSkippedTableId = - if (isComplete) { - "pending" - } else { - "skipped" - } + val pendingOrSkippedTableId = + if (isComplete) { + "pending" + } else { + "skipped" + } - val activeStagesTable = - new StageTableBase(request, activeStages, "active", "activeStage", parent.basePath, - basePath, parent.jobProgresslistener, parent.isFairScheduler, - killEnabled = parent.killEnabled, isFailedStage = false) - val pendingOrSkippedStagesTable = - new StageTableBase(request, pendingOrSkippedStages, pendingOrSkippedTableId, "pendingStage", - parent.basePath, basePath, parent.jobProgresslistener, parent.isFairScheduler, - killEnabled = false, isFailedStage = false) - val completedStagesTable = - new StageTableBase(request, completedStages, "completed", "completedStage", parent.basePath, - basePath, parent.jobProgresslistener, parent.isFairScheduler, - killEnabled = false, isFailedStage = false) - val failedStagesTable = - new StageTableBase(request, failedStages, "failed", "failedStage", parent.basePath, - basePath, parent.jobProgresslistener, parent.isFairScheduler, - killEnabled = false, isFailedStage = true) + val activeStagesTable = + new StageTableBase(store, request, activeStages, "active", "activeStage", parent.basePath, + basePath, parent.isFairScheduler, + killEnabled = parent.killEnabled, isFailedStage = false) + val pendingOrSkippedStagesTable = + new StageTableBase(store, request, pendingOrSkippedStages, pendingOrSkippedTableId, + "pendingStage", parent.basePath, basePath, parent.isFairScheduler, + killEnabled = false, isFailedStage = false) + val completedStagesTable = + new StageTableBase(store, request, completedStages, "completed", "completedStage", + parent.basePath, basePath, parent.isFairScheduler, + killEnabled = false, isFailedStage = false) + val failedStagesTable = + new StageTableBase(store, request, failedStages, "failed", "failedStage", parent.basePath, + basePath, parent.isFairScheduler, + killEnabled = false, isFailedStage = true) - val shouldShowActiveStages = activeStages.nonEmpty - val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty - val shouldShowCompletedStages = completedStages.nonEmpty - val shouldShowSkippedStages = isComplete && pendingOrSkippedStages.nonEmpty - val shouldShowFailedStages = failedStages.nonEmpty + val shouldShowActiveStages = activeStages.nonEmpty + val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowCompletedStages = completedStages.nonEmpty + val shouldShowSkippedStages = isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowFailedStages = failedStages.nonEmpty - val summary: NodeSeq = -
- +
- var content = summary - val appStartTime = listener.startTime - val operationGraphListener = parent.operationGraphListener + var content = summary + val appStartTime = store.applicationInfo().attempts.head.startTime.getTime() - content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, - parent.parent.store.executorList(false), appStartTime) + content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, + store.executorList(false), appStartTime) - content ++= UIUtils.showDagVizForJob( - jobId, operationGraphListener.getOperationGraphForJob(jobId)) + content ++= UIUtils.showDagVizForJob( + jobId, store.operationGraphForJob(jobId)) - if (shouldShowActiveStages) { - content ++=

Active Stages ({activeStages.size})

++ - activeStagesTable.toNodeSeq - } - if (shouldShowPendingStages) { - content ++=

Pending Stages ({pendingOrSkippedStages.size})

++ - pendingOrSkippedStagesTable.toNodeSeq - } - if (shouldShowCompletedStages) { - content ++=

Completed Stages ({completedStages.size})

++ - completedStagesTable.toNodeSeq - } - if (shouldShowSkippedStages) { - content ++=

Skipped Stages ({pendingOrSkippedStages.size})

++ - pendingOrSkippedStagesTable.toNodeSeq - } - if (shouldShowFailedStages) { - content ++=

Failed Stages ({failedStages.size})

++ - failedStagesTable.toNodeSeq - } - UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent, showVisualization = true) + if (shouldShowActiveStages) { + content ++=

Active Stages ({activeStages.size})

++ + activeStagesTable.toNodeSeq + } + if (shouldShowPendingStages) { + content ++=

Pending Stages ({pendingOrSkippedStages.size})

++ + pendingOrSkippedStagesTable.toNodeSeq + } + if (shouldShowCompletedStages) { + content ++=

Completed Stages ({completedStages.size})

++ + completedStagesTable.toNodeSeq + } + if (shouldShowSkippedStages) { + content ++=

Skipped Stages ({pendingOrSkippedStages.size})

++ + pendingOrSkippedStagesTable.toNodeSeq + } + if (shouldShowFailedStages) { + content ++=

Failed Stages ({failedStages.size})

++ + failedStagesTable.toNodeSeq } + UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent, showVisualization = true) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index 81ffe04aca49..d1c498c32b5f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -19,35 +19,52 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest +import scala.collection.JavaConverters._ + +import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils} +import org.apache.spark.status.AppStatusStore +import org.apache.spark.ui._ /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobsTab(val parent: SparkUI) extends SparkUITab(parent, "jobs") { +private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore) + extends SparkUITab(parent, "jobs") { + val sc = parent.sc val killEnabled = parent.killEnabled - val jobProgresslistener = parent.jobProgressListener - val operationGraphListener = parent.operationGraphListener - def isFairScheduler: Boolean = - jobProgresslistener.schedulingMode == Some(SchedulingMode.FAIR) + def isFairScheduler: Boolean = { + val configName = "spark.scheduler.mode" + val config = sc match { + case Some(_sc) => + _sc.conf.getOption(configName) + case _ => + store.environmentInfo().sparkProperties.toMap.get(configName) + } + config.map(SchedulingMode.withName) == Some(SchedulingMode.FAIR) + } def getSparkUser: String = parent.getSparkUser - attachPage(new AllJobsPage(this)) - attachPage(new JobPage(this)) + attachPage(new AllJobsPage(this, store)) + attachPage(new JobPage(this, store)) def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { // stripXSS is called first to remove suspicious characters used in XSS attacks val jobId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) jobId.foreach { id => - if (jobProgresslistener.activeJobs.contains(id)) { - sc.foreach(_.cancelJob(id)) - // Do a quick pause here to give Spark time to kill the job so it shows up as - // killed after the refresh. Note that this will block the serving thread so the - // time should be limited in duration. - Thread.sleep(100) + try { + val job = store.job(id) + if (job.status == JobExecutionStatus.RUNNING) { + sc.foreach(_.cancelJob(id)) + // Do a quick pause here to give Spark time to kill the job so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } catch { + case _: NoSuchElementException => } } } 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 4b8c7b203771..1f125b70e0fb 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 @@ -17,50 +17,49 @@ package org.apache.spark.ui.jobs +import java.util.NoSuchElementException import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.scheduler.StageInfo +import org.apache.spark.status.PoolData +import org.apache.spark.status.api.v1._ import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing specific pool details */ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { - private val sc = parent.sc - private val listener = parent.progressListener def render(request: HttpServletRequest): Seq[Node] = { - listener.synchronized { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val poolName = Option(UIUtils.stripXSS(request.getParameter("poolname"))).map { poolname => - UIUtils.decodeURLParameter(poolname) - }.getOrElse { - throw new IllegalArgumentException(s"Missing poolname parameter") - } + // stripXSS is called first to remove suspicious characters used in XSS attacks + val poolName = Option(UIUtils.stripXSS(request.getParameter("poolname"))).map { poolname => + UIUtils.decodeURLParameter(poolname) + }.getOrElse { + throw new IllegalArgumentException(s"Missing poolname parameter") + } - val poolToActiveStages = listener.poolToActiveStages - val activeStages = poolToActiveStages.get(poolName) match { - case Some(s) => s.values.toSeq - case None => Seq.empty[StageInfo] - } - val shouldShowActiveStages = activeStages.nonEmpty - val activeStagesTable = - new StageTableBase(request, activeStages, "", "activeStage", parent.basePath, "stages/pool", - parent.progressListener, parent.isFairScheduler, parent.killEnabled, - isFailedStage = false) + // For now, pool information is only accessible in live UIs + val pool = parent.sc.flatMap(_.getPoolForName(poolName)).getOrElse { + throw new IllegalArgumentException(s"Unknown pool: $poolName") + } - // For now, pool information is only accessible in live UIs - val pools = sc.map(_.getPoolForName(poolName).getOrElse { - throw new IllegalArgumentException(s"Unknown poolname: $poolName") - }).toSeq - val poolTable = new PoolTable(pools, parent) + val uiPool = try { + parent.store.pool(poolName) + } catch { + case _: NoSuchElementException => + new PoolData(poolName, Set()) + } - var content =

Summary

++ poolTable.toNodeSeq - if (shouldShowActiveStages) { - content ++=

Active Stages ({activeStages.size})

++ activeStagesTable.toNodeSeq - } + val activeStages = uiPool.stageIds.toSeq.map(parent.store.lastStageAttempt(_)) + val activeStagesTable = + new StageTableBase(parent.store, request, activeStages, "", "activeStage", parent.basePath, + "stages/pool", parent.isFairScheduler, parent.killEnabled, false) - UIUtils.headerSparkPage("Fair Scheduler Pool: " + poolName, content, parent) + val poolTable = new PoolTable(Map(pool -> uiPool), parent) + var content =

Summary

++ poolTable.toNodeSeq + if (activeStages.nonEmpty) { + content ++=

Active Stages ({activeStages.size})

++ activeStagesTable.toNodeSeq } + + UIUtils.headerSparkPage("Fair Scheduler Pool: " + poolName, content, parent) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index ea02968733ca..ddc42eea2670 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -22,22 +22,15 @@ import java.net.URLEncoder import scala.collection.mutable.HashMap import scala.xml.Node -import org.apache.spark.scheduler.{Schedulable, StageInfo} +import org.apache.spark.scheduler.Schedulable +import org.apache.spark.status.PoolData +import org.apache.spark.status.api.v1.StageData import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { - private val listener = parent.progressListener +private[ui] class PoolTable(pools: Map[Schedulable, PoolData], parent: StagesTab) { def toNodeSeq: Seq[Node] = { - listener.synchronized { - poolTable(poolRow, pools) - } - } - - private def poolTable( - makeRow: (Schedulable, HashMap[String, HashMap[Int, StageInfo]]) => Seq[Node], - rows: Seq[Schedulable]): Seq[Node] = { @@ -48,29 +41,24 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { - {rows.map(r => makeRow(r, listener.poolToActiveStages))} + {pools.map { case (s, p) => poolRow(s, p) }}
Pool NameSchedulingMode
} - private def poolRow( - p: Schedulable, - poolToActiveStages: HashMap[String, HashMap[Int, StageInfo]]): Seq[Node] = { - val activeStages = poolToActiveStages.get(p.name) match { - case Some(stages) => stages.size - case None => 0 - } + private def poolRow(s: Schedulable, p: PoolData): Seq[Node] = { + val activeStages = p.stageIds.size val href = "%s/stages/pool?poolname=%s" .format(UIUtils.prependBaseUri(parent.basePath), URLEncoder.encode(p.name, "UTF-8")) {p.name} - {p.minShare} - {p.weight} + {s.minShare} + {s.weight} {activeStages} - {p.runningTasks} - {p.schedulingMode} + {s.runningTasks} + {s.schedulingMode} } } 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 3151b8d55465..e3db8adad94f 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 @@ -18,29 +18,28 @@ package org.apache.spark.ui.jobs import java.net.URLEncoder -import java.util.Date +import java.util.{Date, NoSuchElementException} import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.HashSet +import scala.collection.mutable.{HashMap, HashSet} import scala.xml.{Elem, Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.SparkConf -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo, TaskLocality} +import org.apache.spark.internal.config._ +import org.apache.spark.scheduler.{TaskInfo, TaskLocality} import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.api.v1._ import org.apache.spark.ui._ import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.util.{Distribution, Utils} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends WebUIPage("stage") { + import ApiHelper._ import StagePage._ - private val progressListener = parent.progressListener - private val operationGraphListener = parent.operationGraphListener - private val TIMELINE_LEGEND = {
@@ -69,555 +68,526 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We // if we find that it's okay. private val MAX_TIMELINE_TASKS = parent.conf.getInt("spark.ui.timeline.tasks.maximum", 1000) - private def getLocalitySummaryString(stageData: StageUIData): String = { - val localities = stageData.taskData.values.map(_.taskInfo.taskLocality) + private def getLocalitySummaryString(stageData: StageData, taskList: Seq[TaskData]): String = { + val localities = taskList.map(_.taskLocality) val localityCounts = localities.groupBy(identity).mapValues(_.size) + val names = Map( + TaskLocality.PROCESS_LOCAL.toString() -> "Process local", + TaskLocality.NODE_LOCAL.toString() -> "Node local", + TaskLocality.RACK_LOCAL.toString() -> "Rack local", + TaskLocality.ANY.toString() -> "Any") val localityNamesAndCounts = localityCounts.toSeq.map { case (locality, count) => - val localityName = locality match { - case TaskLocality.PROCESS_LOCAL => "Process local" - case TaskLocality.NODE_LOCAL => "Node local" - case TaskLocality.RACK_LOCAL => "Rack local" - case TaskLocality.ANY => "Any" - } - s"$localityName: $count" + s"${names(locality)}: $count" } localityNamesAndCounts.sorted.mkString("; ") } def render(request: HttpServletRequest): Seq[Node] = { - progressListener.synchronized { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) - require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - - val parameterAttempt = UIUtils.stripXSS(request.getParameter("attempt")) - require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") - - val parameterTaskPage = UIUtils.stripXSS(request.getParameter("task.page")) - val parameterTaskSortColumn = UIUtils.stripXSS(request.getParameter("task.sort")) - val parameterTaskSortDesc = UIUtils.stripXSS(request.getParameter("task.desc")) - val parameterTaskPageSize = UIUtils.stripXSS(request.getParameter("task.pageSize")) - val parameterTaskPrevPageSize = UIUtils.stripXSS(request.getParameter("task.prevPageSize")) - - val taskPage = Option(parameterTaskPage).map(_.toInt).getOrElse(1) - val taskSortColumn = Option(parameterTaskSortColumn).map { sortColumn => - UIUtils.decodeURLParameter(sortColumn) - }.getOrElse("Index") - val taskSortDesc = Option(parameterTaskSortDesc).map(_.toBoolean).getOrElse(false) - val taskPageSize = Option(parameterTaskPageSize).map(_.toInt).getOrElse(100) - val taskPrevPageSize = Option(parameterTaskPrevPageSize).map(_.toInt).getOrElse(taskPageSize) - - val stageId = parameterId.toInt - val stageAttemptId = parameterAttempt.toInt - val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId)) - - val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" - if (stageDataOption.isEmpty) { + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + + val parameterAttempt = UIUtils.stripXSS(request.getParameter("attempt")) + require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") + + val parameterTaskPage = UIUtils.stripXSS(request.getParameter("task.page")) + val parameterTaskSortColumn = UIUtils.stripXSS(request.getParameter("task.sort")) + val parameterTaskSortDesc = UIUtils.stripXSS(request.getParameter("task.desc")) + val parameterTaskPageSize = UIUtils.stripXSS(request.getParameter("task.pageSize")) + val parameterTaskPrevPageSize = UIUtils.stripXSS(request.getParameter("task.prevPageSize")) + + val taskPage = Option(parameterTaskPage).map(_.toInt).getOrElse(1) + val taskSortColumn = Option(parameterTaskSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("Index") + val taskSortDesc = Option(parameterTaskSortDesc).map(_.toBoolean).getOrElse(false) + val taskPageSize = Option(parameterTaskPageSize).map(_.toInt).getOrElse(100) + val taskPrevPageSize = Option(parameterTaskPrevPageSize).map(_.toInt).getOrElse(taskPageSize) + + val stageId = parameterId.toInt + val stageAttemptId = parameterAttempt.toInt + + val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" + val stageData = try { + parent.store.stageAttempt(stageId, stageAttemptId, details = true) + } catch { + case _: NoSuchElementException => val content =

No information to display for Stage {stageId} (Attempt {stageAttemptId})

return UIUtils.headerSparkPage(stageHeader, content, parent) + } - } - if (stageDataOption.get.taskData.isEmpty) { - val content = -
-

Summary Metrics

No tasks have started yet -

Tasks

No tasks have started yet -
- return UIUtils.headerSparkPage(stageHeader, content, parent) - } - - val stageData = stageDataOption.get - val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime) - val numCompleted = stageData.numCompleteTasks - val totalTasks = stageData.numActiveTasks + - stageData.numCompleteTasks + stageData.numFailedTasks - val totalTasksNumStr = if (totalTasks == tasks.size) { - s"$totalTasks" - } else { - s"$totalTasks, showing ${tasks.size}" - } + val tasks = stageData.tasks.getOrElse(Map.empty).values.toSeq + if (tasks.isEmpty) { + val content = +
+

Summary Metrics

No tasks have started yet +

Tasks

No tasks have started yet +
+ return UIUtils.headerSparkPage(stageHeader, content, parent) + } - val allAccumulables = progressListener.stageIdToData((stageId, stageAttemptId)).accumulables - val externalAccumulables = allAccumulables.values.filter { acc => !acc.internal } - val hasAccumulators = externalAccumulables.nonEmpty + val numCompleted = stageData.numCompleteTasks + val totalTasks = stageData.numActiveTasks + stageData.numCompleteTasks + + stageData.numFailedTasks + stageData.numKilledTasks + val totalTasksNumStr = if (totalTasks == tasks.size) { + s"$totalTasks" + } else { + s"$totalTasks, showing ${tasks.size}" + } - val summary = -
-
    + val externalAccumulables = stageData.accumulatorUpdates + val hasAccumulators = externalAccumulables.size > 0 + + val summary = +
    +
      +
    • + Total Time Across All Tasks: + {UIUtils.formatDuration(stageData.executorRunTime)} +
    • +
    • + Locality Level Summary: + {getLocalitySummaryString(stageData, tasks)} +
    • + {if (hasInput(stageData)) {
    • - Total Time Across All Tasks: - {UIUtils.formatDuration(stageData.executorRunTime)} + Input Size / Records: + {s"${Utils.bytesToString(stageData.inputBytes)} / ${stageData.inputRecords}"}
    • + }} + {if (hasOutput(stageData)) {
    • - Locality Level Summary: - {getLocalitySummaryString(stageData)} + Output: + {s"${Utils.bytesToString(stageData.outputBytes)} / ${stageData.outputRecords}"}
    • - {if (stageData.hasInput) { -
    • - Input Size / Records: - {s"${Utils.bytesToString(stageData.inputBytes)} / ${stageData.inputRecords}"} -
    • - }} - {if (stageData.hasOutput) { -
    • - Output: - {s"${Utils.bytesToString(stageData.outputBytes)} / ${stageData.outputRecords}"} -
    • - }} - {if (stageData.hasShuffleRead) { -
    • - Shuffle Read: - {s"${Utils.bytesToString(stageData.shuffleReadTotalBytes)} / " + - s"${stageData.shuffleReadRecords}"} -
    • - }} - {if (stageData.hasShuffleWrite) { -
    • - Shuffle Write: - {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + - s"${stageData.shuffleWriteRecords}"} -
    • - }} - {if (stageData.hasBytesSpilled) { -
    • - Shuffle Spill (Memory): - {Utils.bytesToString(stageData.memoryBytesSpilled)} -
    • -
    • - Shuffle Spill (Disk): - {Utils.bytesToString(stageData.diskBytesSpilled)} -
    • - }} -
    -
    + }} + {if (hasShuffleRead(stageData)) { +
  • + Shuffle Read: + {s"${Utils.bytesToString(stageData.shuffleReadBytes)} / " + + s"${stageData.shuffleReadRecords}"} +
  • + }} + {if (hasShuffleWrite(stageData)) { +
  • + Shuffle Write: + {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + + s"${stageData.shuffleWriteRecords}"} +
  • + }} + {if (hasBytesSpilled(stageData)) { +
  • + Shuffle Spill (Memory): + {Utils.bytesToString(stageData.memoryBytesSpilled)} +
  • +
  • + Shuffle Spill (Disk): + {Utils.bytesToString(stageData.diskBytesSpilled)} +
  • + }} +
+
- val showAdditionalMetrics = -
- - - Show Additional Metrics - - +
- val outputSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.outputMetrics.bytesWritten.toDouble - } + val stageGraph = try { + Some(parent.store.operationGraphForStage(stageId)) + } catch { + case _: NoSuchElementException => None + } + val dagViz = UIUtils.showDagVizForStage(stageId, stageGraph) - val outputRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.outputMetrics.recordsWritten.toDouble - } + val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") + def accumulableRow(acc: AccumulableInfo): Seq[Node] = { + {acc.name}{acc.value} + } + val accumulableTable = UIUtils.listingTable( + accumulableHeaders, + accumulableRow, + externalAccumulables.toSeq) + + val page: Int = { + // If the user has changed to a larger page size, then go to page 1 in order to avoid + // IndexOutOfBoundsException. + if (taskPageSize <= taskPrevPageSize) { + taskPage + } else { + 1 + } + } + val currentTime = System.currentTimeMillis() + val (taskTable, taskTableHTML) = try { + val _taskTable = new TaskPagedTable( + parent.conf, + UIUtils.prependBaseUri(parent.basePath) + + s"/stages/stage?id=${stageId}&attempt=${stageAttemptId}", + tasks, + hasAccumulators, + hasInput(stageData), + hasOutput(stageData), + hasShuffleRead(stageData), + hasShuffleWrite(stageData), + hasBytesSpilled(stageData), + currentTime, + pageSize = taskPageSize, + sortColumn = taskSortColumn, + desc = taskSortDesc, + store = parent.store + ) + (_taskTable, _taskTable.table(page)) + } catch { + case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) => + val errorMessage = +
+

Error while rendering stage table:

+
+              {Utils.exceptionString(e)}
+            
+
+ (null, errorMessage) + } - val outputQuantiles = Output Size / Records +: - getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) + val jsForScrollingDownToTaskTable = + - val shuffleReadBlockedTimes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.fetchWaitTime.toDouble - } - val shuffleReadBlockedQuantiles = - - - Shuffle Read Blocked Time - - +: - getFormattedTimeQuantiles(shuffleReadBlockedTimes) + val taskIdsInPage = if (taskTable == null) Set.empty[Long] + else taskTable.dataSource.slicedTaskIds - val shuffleReadTotalSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.totalBytesRead.toDouble - } - val shuffleReadTotalRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.recordsRead.toDouble - } - val shuffleReadTotalQuantiles = - - - Shuffle Read Size / Records - - +: - getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) + // Excludes tasks which failed and have incomplete metrics + val validTasks = tasks.filter(t => t.status == "SUCCESS" && t.taskMetrics.isDefined) - val shuffleReadRemoteSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleReadMetrics.remoteBytesRead.toDouble + val summaryTable: Option[Seq[Node]] = + if (validTasks.size == 0) { + None + } else { + def getDistributionQuantiles(data: Seq[Double]): IndexedSeq[Double] = { + Distribution(data).get.getQuantiles() + } + def getFormattedTimeQuantiles(times: Seq[Double]): Seq[Node] = { + getDistributionQuantiles(times).map { millis => + {UIUtils.formatDuration(millis.toLong)} } - val shuffleReadRemoteQuantiles = - - - Shuffle Remote Reads - - +: - getFormattedSizeQuantiles(shuffleReadRemoteSizes) + } + def getFormattedSizeQuantiles(data: Seq[Double]): Seq[Elem] = { + getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)}) + } - val shuffleWriteSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleWriteMetrics.bytesWritten.toDouble - } + val deserializationTimes = validTasks.map { task => + task.taskMetrics.get.executorDeserializeTime.toDouble + } + val deserializationQuantiles = + + + Task Deserialization Time + + +: getFormattedTimeQuantiles(deserializationTimes) + + val serviceTimes = validTasks.map(_.taskMetrics.get.executorRunTime.toDouble) + val serviceQuantiles = Duration +: getFormattedTimeQuantiles(serviceTimes) + + val gcTimes = validTasks.map(_.taskMetrics.get.jvmGcTime.toDouble) + val gcQuantiles = + + GC Time + + +: getFormattedTimeQuantiles(gcTimes) + + val serializationTimes = validTasks.map(_.taskMetrics.get.resultSerializationTime.toDouble) + val serializationQuantiles = + + + Result Serialization Time + + +: getFormattedTimeQuantiles(serializationTimes) + + val gettingResultTimes = validTasks.map(getGettingResultTime(_, currentTime).toDouble) + val gettingResultQuantiles = + + + Getting Result Time + + +: + getFormattedTimeQuantiles(gettingResultTimes) + + val peakExecutionMemory = validTasks.map(_.taskMetrics.get.peakExecutionMemory.toDouble) + val peakExecutionMemoryQuantiles = { + + + Peak Execution Memory + + +: getFormattedSizeQuantiles(peakExecutionMemory) + } - val shuffleWriteRecords = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.shuffleWriteMetrics.recordsWritten.toDouble - } + // The scheduler delay includes the network delay to send the task to the worker + // machine and to send back the result (but not the time to fetch the task result, + // if it needed to be fetched from the block manager on the worker). + val schedulerDelays = validTasks.map { task => + getSchedulerDelay(task, task.taskMetrics.get, currentTime).toDouble + } + val schedulerDelayTitle = Scheduler Delay + val schedulerDelayQuantiles = schedulerDelayTitle +: + getFormattedTimeQuantiles(schedulerDelays) + def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) + : Seq[Elem] = { + val recordDist = getDistributionQuantiles(records).iterator + getDistributionQuantiles(data).map(d => + {s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"} + ) + } - val shuffleWriteQuantiles = Shuffle Write Size / Records +: - getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) + val inputSizes = validTasks.map(_.taskMetrics.get.inputMetrics.bytesRead.toDouble) + val inputRecords = validTasks.map(_.taskMetrics.get.inputMetrics.recordsRead.toDouble) + val inputQuantiles = Input Size / Records +: + getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords) - val memoryBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.memoryBytesSpilled.toDouble - } - val memoryBytesSpilledQuantiles = Shuffle spill (memory) +: - getFormattedSizeQuantiles(memoryBytesSpilledSizes) + val outputSizes = validTasks.map(_.taskMetrics.get.outputMetrics.bytesWritten.toDouble) + val outputRecords = validTasks.map(_.taskMetrics.get.outputMetrics.recordsWritten.toDouble) + val outputQuantiles = Output Size / Records +: + getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) - val diskBytesSpilledSizes = validTasks.map { taskUIData: TaskUIData => - taskUIData.metrics.get.diskBytesSpilled.toDouble - } - val diskBytesSpilledQuantiles = Shuffle spill (disk) +: - getFormattedSizeQuantiles(diskBytesSpilledSizes) - - val listings: Seq[Seq[Node]] = Seq( - {serviceQuantiles}, - {schedulerDelayQuantiles}, - - {deserializationQuantiles} - - {gcQuantiles}, - - {serializationQuantiles} - , - {gettingResultQuantiles}, - - {peakExecutionMemoryQuantiles} - , - if (stageData.hasInput) {inputQuantiles} else Nil, - if (stageData.hasOutput) {outputQuantiles} else Nil, - if (stageData.hasShuffleRead) { - - {shuffleReadBlockedQuantiles} - - {shuffleReadTotalQuantiles} - - {shuffleReadRemoteQuantiles} - - } else { - Nil - }, - if (stageData.hasShuffleWrite) {shuffleWriteQuantiles} else Nil, - if (stageData.hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil, - if (stageData.hasBytesSpilled) {diskBytesSpilledQuantiles} else Nil) - - val quantileHeaders = Seq("Metric", "Min", "25th percentile", - "Median", "75th percentile", "Max") - // The summary table does not use CSS to stripe rows, which doesn't work with hidden - // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows). - Some(UIUtils.listingTable( - quantileHeaders, - identity[Seq[Node]], - listings, - fixedWidth = true, - id = Some("task-summary-table"), - stripeRowsWithCss = false)) + val shuffleReadBlockedTimes = validTasks.map { task => + task.taskMetrics.get.shuffleReadMetrics.fetchWaitTime.toDouble + } + val shuffleReadBlockedQuantiles = + + + Shuffle Read Blocked Time + + +: + getFormattedTimeQuantiles(shuffleReadBlockedTimes) + + val shuffleReadTotalSizes = validTasks.map { task => + totalBytesRead(task.taskMetrics.get.shuffleReadMetrics).toDouble + } + val shuffleReadTotalRecords = validTasks.map { task => + task.taskMetrics.get.shuffleReadMetrics.recordsRead.toDouble + } + val shuffleReadTotalQuantiles = + + + Shuffle Read Size / Records + + +: + getFormattedSizeQuantilesWithRecords(shuffleReadTotalSizes, shuffleReadTotalRecords) + + val shuffleReadRemoteSizes = validTasks.map { task => + task.taskMetrics.get.shuffleReadMetrics.remoteBytesRead.toDouble + } + val shuffleReadRemoteQuantiles = + + + Shuffle Remote Reads + + +: + getFormattedSizeQuantiles(shuffleReadRemoteSizes) + + val shuffleWriteSizes = validTasks.map { task => + task.taskMetrics.get.shuffleWriteMetrics.bytesWritten.toDouble } - val executorTable = new ExecutorTable(stageId, stageAttemptId, parent, store) + val shuffleWriteRecords = validTasks.map { task => + task.taskMetrics.get.shuffleWriteMetrics.recordsWritten.toDouble + } - val maybeAccumulableTable: Seq[Node] = - if (hasAccumulators) {

Accumulators

++ accumulableTable } else Seq.empty + val shuffleWriteQuantiles = Shuffle Write Size / Records +: + getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) + + val memoryBytesSpilledSizes = validTasks.map(_.taskMetrics.get.memoryBytesSpilled.toDouble) + val memoryBytesSpilledQuantiles = Shuffle spill (memory) +: + getFormattedSizeQuantiles(memoryBytesSpilledSizes) + + val diskBytesSpilledSizes = validTasks.map(_.taskMetrics.get.diskBytesSpilled.toDouble) + val diskBytesSpilledQuantiles = Shuffle spill (disk) +: + getFormattedSizeQuantiles(diskBytesSpilledSizes) + + val listings: Seq[Seq[Node]] = Seq( + {serviceQuantiles}, + {schedulerDelayQuantiles}, + + {deserializationQuantiles} + + {gcQuantiles}, + + {serializationQuantiles} + , + {gettingResultQuantiles}, + + {peakExecutionMemoryQuantiles} + , + if (hasInput(stageData)) {inputQuantiles} else Nil, + if (hasOutput(stageData)) {outputQuantiles} else Nil, + if (hasShuffleRead(stageData)) { + + {shuffleReadBlockedQuantiles} + + {shuffleReadTotalQuantiles} + + {shuffleReadRemoteQuantiles} + + } else { + Nil + }, + if (hasShuffleWrite(stageData)) {shuffleWriteQuantiles} else Nil, + if (hasBytesSpilled(stageData)) {memoryBytesSpilledQuantiles} else Nil, + if (hasBytesSpilled(stageData)) {diskBytesSpilledQuantiles} else Nil) + + val quantileHeaders = Seq("Metric", "Min", "25th percentile", + "Median", "75th percentile", "Max") + // The summary table does not use CSS to stripe rows, which doesn't work with hidden + // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows). + Some(UIUtils.listingTable( + quantileHeaders, + identity[Seq[Node]], + listings, + fixedWidth = true, + id = Some("task-summary-table"), + stripeRowsWithCss = false)) + } - val aggMetrics = - -

- - Aggregated Metrics by Executor -

-
-
- {executorTable.toNodeSeq} -
+ val executorTable = new ExecutorTable(stageData, parent.store) + + val maybeAccumulableTable: Seq[Node] = + if (hasAccumulators) {

Accumulators

++ accumulableTable } else Seq() + + val aggMetrics = + +

+ + Aggregated Metrics by Executor +

+
+
+ {executorTable.toNodeSeq} +
- val content = - summary ++ - dagViz ++ - showAdditionalMetrics ++ - makeTimeline( - // Only show the tasks in the table - stageData.taskData.values.toSeq.filter(t => taskIdsInPage.contains(t.taskInfo.taskId)), - currentTime) ++ -

Summary Metrics for {numCompleted} Completed Tasks

++ -
{summaryTable.getOrElse("No tasks have reported metrics yet.")}
++ - aggMetrics ++ - maybeAccumulableTable ++ -

Tasks ({totalTasksNumStr})

++ - taskTableHTML ++ jsForScrollingDownToTaskTable - UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) - } + val content = + summary ++ + dagViz ++ + showAdditionalMetrics ++ + makeTimeline( + // Only show the tasks in the table + tasks.filter { t => taskIdsInPage.contains(t.taskId) }, + currentTime) ++ +

Summary Metrics for {numCompleted} Completed Tasks

++ +
{summaryTable.getOrElse("No tasks have reported metrics yet.")}
++ + aggMetrics ++ + maybeAccumulableTable ++ +

Tasks ({totalTasksNumStr})

++ + taskTableHTML ++ jsForScrollingDownToTaskTable + UIUtils.headerSparkPage(stageHeader, content, parent, showVisualization = true) } - def makeTimeline(tasks: Seq[TaskUIData], currentTime: Long): Seq[Node] = { + def makeTimeline(tasks: Seq[TaskData], currentTime: Long): Seq[Node] = { val executorsSet = new HashSet[(String, String)] var minLaunchTime = Long.MaxValue var maxFinishTime = Long.MinValue val executorsArrayStr = - tasks.sortBy(-_.taskInfo.launchTime).take(MAX_TIMELINE_TASKS).map { taskUIData => - val taskInfo = taskUIData.taskInfo + tasks.sortBy(-_.launchTime.getTime()).take(MAX_TIMELINE_TASKS).map { taskInfo => val executorId = taskInfo.executorId val host = taskInfo.host executorsSet += ((executorId, host)) - val launchTime = taskInfo.launchTime - val finishTime = if (!taskInfo.running) taskInfo.finishTime else currentTime + val launchTime = taskInfo.launchTime.getTime() + val finishTime = taskInfo.duration.map(taskInfo.launchTime.getTime() + _) + .getOrElse(currentTime) val totalExecutionTime = finishTime - launchTime minLaunchTime = launchTime.min(minLaunchTime) maxFinishTime = finishTime.max(maxFinishTime) def toProportion(time: Long) = time.toDouble / totalExecutionTime * 100 - val metricsOpt = taskUIData.metrics + val metricsOpt = taskInfo.taskMetrics val shuffleReadTime = metricsOpt.map(_.shuffleReadMetrics.fetchWaitTime).getOrElse(0L) val shuffleReadTimeProportion = toProportion(shuffleReadTime) @@ -629,14 +599,14 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val serializationTimeProportion = toProportion(serializationTime) val deserializationTime = metricsOpt.map(_.executorDeserializeTime).getOrElse(0L) val deserializationTimeProportion = toProportion(deserializationTime) - val gettingResultTime = getGettingResultTime(taskUIData.taskInfo, currentTime) + val gettingResultTime = getGettingResultTime(taskInfo, currentTime) val gettingResultTimeProportion = toProportion(gettingResultTime) val schedulerDelay = metricsOpt.map(getSchedulerDelay(taskInfo, _, currentTime)).getOrElse(0L) val schedulerDelayProportion = toProportion(schedulerDelay) val executorOverhead = serializationTime + deserializationTime - val executorRunTime = if (taskInfo.running) { + val executorRunTime = if (taskInfo.duration.isDefined) { totalExecutionTime - executorOverhead - gettingResultTime } else { metricsOpt.map(_.executorRunTime).getOrElse( @@ -663,7 +633,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We serializationTimeProportionPos + serializationTimeProportion val index = taskInfo.index - val attempt = taskInfo.attemptNumber + val attempt = taskInfo.attempt val svgTag = if (totalExecutionTime == 0) { @@ -705,7 +675,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We |Status: ${taskInfo.status}
|Launch Time: ${UIUtils.formatDate(new Date(launchTime))} |${ - if (!taskInfo.running) { + if (!taskInfo.duration.isDefined) { s"""
Finish Time: ${UIUtils.formatDate(new Date(finishTime))}""" } else { "" @@ -770,34 +740,40 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We } private[ui] object StagePage { - private[ui] def getGettingResultTime(info: TaskInfo, currentTime: Long): Long = { - if (info.gettingResult) { - if (info.finished) { - info.finishTime - info.gettingResultTime - } else { - // The task is still fetching the result. - currentTime - info.gettingResultTime - } - } else { - 0L + private[ui] def getGettingResultTime(info: TaskData, currentTime: Long): Long = { + info.resultFetchStart match { + case Some(start) => + info.duration match { + case Some(duration) => + info.launchTime.getTime() + duration - start.getTime() + + case _ => + currentTime - start.getTime() + } + + case _ => + 0L } } private[ui] def getSchedulerDelay( - info: TaskInfo, metrics: TaskMetricsUIData, currentTime: Long): Long = { - if (info.finished) { - val totalExecutionTime = info.finishTime - info.launchTime - val executorOverhead = metrics.executorDeserializeTime + - metrics.resultSerializationTime - math.max( - 0, - totalExecutionTime - metrics.executorRunTime - executorOverhead - - getGettingResultTime(info, currentTime)) - } else { - // The task is still running and the metrics like executorRunTime are not available. - 0L + info: TaskData, + metrics: TaskMetrics, + currentTime: Long): Long = { + info.duration match { + case Some(duration) => + val executorOverhead = metrics.executorDeserializeTime + metrics.resultSerializationTime + math.max( + 0, + duration - metrics.executorRunTime - executorOverhead - + getGettingResultTime(info, currentTime)) + + case _ => + // The task is still running and the metrics like executorRunTime are not available. + 0L } } + } private[ui] case class TaskTableRowInputData(inputSortable: Long, inputReadable: String) @@ -826,7 +802,7 @@ private[ui] case class TaskTableRowBytesSpilledData( /** * Contains all data that needs for sorting and generating HTML. Using this one rather than - * TaskUIData to avoid creating duplicate contents during sorting the data. + * TaskData to avoid creating duplicate contents during sorting the data. */ private[ui] class TaskTableRowData( val index: Int, @@ -856,14 +832,13 @@ private[ui] class TaskTableRowData( val logs: Map[String, String]) private[ui] class TaskDataSource( - tasks: Seq[TaskUIData], + tasks: Seq[TaskData], hasAccumulators: Boolean, hasInput: Boolean, hasOutput: Boolean, hasShuffleRead: Boolean, hasShuffleWrite: Boolean, hasBytesSpilled: Boolean, - lastUpdateTime: Option[Long], currentTime: Long, pageSize: Int, sortColumn: String, @@ -871,7 +846,10 @@ private[ui] class TaskDataSource( store: AppStatusStore) extends PagedDataSource[TaskTableRowData](pageSize) { import StagePage._ - // Convert TaskUIData to TaskTableRowData which contains the final contents to show in the table + // Keep an internal cache of executor log maps so that long task lists render faster. + private val executors = new HashMap[String, Map[String, String]]() + + // Convert TaskData to TaskTableRowData which contains the final contents to show in the table // so that we can avoid creating duplicate contents during sorting the data private val data = tasks.map(taskRow).sorted(ordering(sortColumn, desc)) @@ -887,26 +865,19 @@ private[ui] class TaskDataSource( def slicedTaskIds: Set[Long] = _slicedTaskIds - private def taskRow(taskData: TaskUIData): TaskTableRowData = { - val info = taskData.taskInfo - val metrics = taskData.metrics - val duration = taskData.taskDuration(lastUpdateTime).getOrElse(1L) - val formatDuration = - taskData.taskDuration(lastUpdateTime).map(d => UIUtils.formatDuration(d)).getOrElse("") + private def taskRow(info: TaskData): TaskTableRowData = { + val metrics = info.taskMetrics + val duration = info.duration.getOrElse(1L) + val formatDuration = info.duration.map(d => UIUtils.formatDuration(d)).getOrElse("") val schedulerDelay = metrics.map(getSchedulerDelay(info, _, currentTime)).getOrElse(0L) - val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) + val gcTime = metrics.map(_.jvmGcTime).getOrElse(0L) val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = getGettingResultTime(info, currentTime) - val externalAccumulableReadable = info.accumulables - .filterNot(_.internal) - .flatMap { a => - (a.name, a.update) match { - case (Some(name), Some(update)) => Some(StringEscapeUtils.escapeHtml4(s"$name: $update")) - case _ => None - } - } + val externalAccumulableReadable = info.accumulatorUpdates.map { acc => + StringEscapeUtils.escapeHtml4(s"${acc.name}: ${acc.update}") + } val peakExecutionMemoryUsed = metrics.map(_.peakExecutionMemory).getOrElse(0L) val maybeInput = metrics.map(_.inputMetrics) @@ -928,7 +899,7 @@ private[ui] class TaskDataSource( val shuffleReadBlockedTimeReadable = maybeShuffleRead.map(ms => UIUtils.formatDuration(ms.fetchWaitTime)).getOrElse("") - val totalShuffleBytes = maybeShuffleRead.map(_.totalBytesRead) + val totalShuffleBytes = maybeShuffleRead.map(ApiHelper.totalBytesRead) val shuffleReadSortable = totalShuffleBytes.getOrElse(0L) val shuffleReadReadable = totalShuffleBytes.map(Utils.bytesToString).getOrElse("") val shuffleReadRecords = maybeShuffleRead.map(_.recordsRead.toString).getOrElse("") @@ -1011,17 +982,16 @@ private[ui] class TaskDataSource( None } - val logs = store.executorSummary(info.executorId).map(_.executorLogs).getOrElse(Map.empty) new TaskTableRowData( info.index, info.taskId, - info.attemptNumber, + info.attempt, info.speculative, info.status, info.taskLocality.toString, info.executorId, info.host, - info.launchTime, + info.launchTime.getTime(), duration, formatDuration, schedulerDelay, @@ -1036,8 +1006,13 @@ private[ui] class TaskDataSource( shuffleRead, shuffleWrite, bytesSpilled, - taskData.errorMessage.getOrElse(""), - logs) + info.errorMessage.getOrElse(""), + executorLogs(info.executorId)) + } + + private def executorLogs(id: String): Map[String, String] = { + executors.getOrElseUpdate(id, + store.executorSummary(id).map(_.executorLogs).getOrElse(Map.empty)) } /** @@ -1148,14 +1123,13 @@ private[ui] class TaskDataSource( private[ui] class TaskPagedTable( conf: SparkConf, basePath: String, - data: Seq[TaskUIData], + data: Seq[TaskData], hasAccumulators: Boolean, hasInput: Boolean, hasOutput: Boolean, hasShuffleRead: Boolean, hasShuffleWrite: Boolean, hasBytesSpilled: Boolean, - lastUpdateTime: Option[Long], currentTime: Long, pageSize: Int, sortColumn: String, @@ -1181,7 +1155,6 @@ private[ui] class TaskPagedTable( hasShuffleRead, hasShuffleWrite, hasBytesSpilled, - lastUpdateTime, currentTime, pageSize, sortColumn, @@ -1363,3 +1336,23 @@ private[ui] class TaskPagedTable( {errorSummary}{details} } } + +private object ApiHelper { + + def hasInput(stageData: StageData): Boolean = stageData.inputBytes > 0 + + def hasOutput(stageData: StageData): Boolean = stageData.outputBytes > 0 + + def hasShuffleRead(stageData: StageData): Boolean = stageData.shuffleReadBytes > 0 + + def hasShuffleWrite(stageData: StageData): Boolean = stageData.shuffleWriteBytes > 0 + + def hasBytesSpilled(stageData: StageData): Boolean = { + stageData.diskBytesSpilled > 0 || stageData.memoryBytesSpilled > 0 + } + + def totalBytesRead(metrics: ShuffleReadMetrics): Long = { + metrics.localBytesRead + metrics.remoteBytesRead + } + +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index f0a12a28de06..5a488e61da29 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui.jobs +import java.lang.{Long => JLong} import java.net.URLEncoder import java.util.Date import javax.servlet.http.HttpServletRequest @@ -27,18 +28,21 @@ import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.scheduler.StageInfo +import org.apache.spark.status.{AppStatusStore, TaskDataWrapper} +import org.apache.spark.status.api.v1 +import org.apache.spark.storage.RDDInfo import org.apache.spark.ui._ import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils private[ui] class StageTableBase( + store: AppStatusStore, request: HttpServletRequest, - stages: Seq[StageInfo], + stages: Seq[v1.StageData], tableHeaderID: String, stageTag: String, basePath: String, subPath: String, - progressListener: JobProgressListener, isFairScheduler: Boolean, killEnabled: Boolean, isFailedStage: Boolean) { @@ -79,12 +83,12 @@ private[ui] class StageTableBase( val toNodeSeq = try { new StagePagedTable( + store, stages, tableHeaderID, stageTag, basePath, subPath, - progressListener, isFairScheduler, killEnabled, currentTime, @@ -106,13 +110,13 @@ private[ui] class StageTableBase( } private[ui] class StageTableRowData( - val stageInfo: StageInfo, - val stageData: Option[StageUIData], + val stage: v1.StageData, + val option: Option[v1.StageData], val stageId: Int, val attemptId: Int, val schedulingPool: String, val descriptionOption: Option[String], - val submissionTime: Long, + val submissionTime: Date, val formattedSubmissionTime: String, val duration: Long, val formattedDuration: String, @@ -126,19 +130,20 @@ private[ui] class StageTableRowData( val shuffleWriteWithUnit: String) private[ui] class MissingStageTableRowData( - stageInfo: StageInfo, + stageInfo: v1.StageData, stageId: Int, attemptId: Int) extends StageTableRowData( - stageInfo, None, stageId, attemptId, "", None, 0, "", -1, "", 0, "", 0, "", 0, "", 0, "") + stageInfo, None, stageId, attemptId, "", None, new Date(0), "", -1, "", 0, "", 0, "", 0, "", 0, + "") /** Page showing list of all ongoing and recently finished stages */ private[ui] class StagePagedTable( - stages: Seq[StageInfo], + store: AppStatusStore, + stages: Seq[v1.StageData], tableHeaderId: String, stageTag: String, basePath: String, subPath: String, - listener: JobProgressListener, isFairScheduler: Boolean, killEnabled: Boolean, currentTime: Long, @@ -164,8 +169,8 @@ private[ui] class StagePagedTable( parameterOtherTable.mkString("&") override val dataSource = new StageDataSource( + store, stages, - listener, currentTime, pageSize, sortColumn, @@ -274,10 +279,10 @@ private[ui] class StagePagedTable( } private def rowContent(data: StageTableRowData): Seq[Node] = { - data.stageData match { + data.option match { case None => missingStageRow(data.stageId) case Some(stageData) => - val info = data.stageInfo + val info = data.stage {if (data.attemptId > 0) { {data.stageId} (retry {data.attemptId}) @@ -301,8 +306,8 @@ private[ui] class StagePagedTable( {data.formattedDuration} {UIUtils.makeProgressBar(started = stageData.numActiveTasks, - completed = stageData.completedIndices.size, failed = stageData.numFailedTasks, - skipped = 0, reasonToNumKilled = stageData.reasonToNumKilled, total = info.numTasks)} + completed = stageData.numCompleteTasks, failed = stageData.numFailedTasks, + skipped = 0, reasonToNumKilled = stageData.killedTasksSummary, total = info.numTasks)} {data.inputReadWithUnit} {data.outputWriteWithUnit} @@ -318,7 +323,7 @@ private[ui] class StagePagedTable( } } - private def failureReasonHtml(s: StageInfo): Seq[Node] = { + private def failureReasonHtml(s: v1.StageData): Seq[Node] = { val failureReason = s.failureReason.getOrElse("") val isMultiline = failureReason.indexOf('\n') >= 0 // Display the first line by default @@ -344,7 +349,7 @@ private[ui] class StagePagedTable( {failureReasonSummary}{details} } - private def makeDescription(s: StageInfo, descriptionOption: Option[String]): Seq[Node] = { + private def makeDescription(s: v1.StageData, descriptionOption: Option[String]): Seq[Node] = { val basePathUri = UIUtils.prependBaseUri(basePath) val killLink = if (killEnabled) { @@ -368,8 +373,8 @@ private[ui] class StagePagedTable( val nameLinkUri = s"$basePathUri/stages/stage?id=${s.stageId}&attempt=${s.attemptId}" val nameLink = {s.name} - val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) - val details = if (s.details.nonEmpty) { + val cachedRddInfos = store.rddList().filter { rdd => s.rddIds.contains(rdd.id) } + val details = if (s.details != null && s.details.nonEmpty) { +details @@ -404,14 +409,14 @@ private[ui] class StagePagedTable( } private[ui] class StageDataSource( - stages: Seq[StageInfo], - listener: JobProgressListener, + store: AppStatusStore, + stages: Seq[v1.StageData], currentTime: Long, pageSize: Int, sortColumn: String, desc: Boolean) extends PagedDataSource[StageTableRowData](pageSize) { - // Convert StageInfo to StageTableRowData which contains the final contents to show in the table - // so that we can avoid creating duplicate contents during sorting the data + // Convert v1.StageData to StageTableRowData which contains the final contents to show in the + // table so that we can avoid creating duplicate contents during sorting the data private val data = stages.map(stageRow).sorted(ordering(sortColumn, desc)) private var _slicedStageIds: Set[Int] = _ @@ -424,57 +429,46 @@ private[ui] class StageDataSource( r } - private def stageRow(s: StageInfo): StageTableRowData = { - val stageDataOption = listener.stageIdToData.get((s.stageId, s.attemptId)) + private def stageRow(stageData: v1.StageData): StageTableRowData = { + val description = stageData.description.getOrElse("") - if (stageDataOption.isEmpty) { - return new MissingStageTableRowData(s, s.stageId, s.attemptId) - } - val stageData = stageDataOption.get - - val description = stageData.description - - val formattedSubmissionTime = s.submissionTime match { - case Some(t) => UIUtils.formatDate(new Date(t)) + val formattedSubmissionTime = stageData.submissionTime match { + case Some(t) => UIUtils.formatDate(t) case None => "Unknown" } - val finishTime = s.completionTime.getOrElse(currentTime) + val finishTime = stageData.completionTime.map(_.getTime()).getOrElse(currentTime) // The submission time for a stage is misleading because it counts the time // the stage waits to be launched. (SPARK-10930) - val taskLaunchTimes = - stageData.taskData.values.map(_.taskInfo.launchTime).filter(_ > 0) - val duration: Option[Long] = - if (taskLaunchTimes.nonEmpty) { - val startTime = taskLaunchTimes.min - if (finishTime > startTime) { - Some(finishTime - startTime) - } else { - Some(currentTime - startTime) - } + val duration = stageData.firstTaskLaunchedTime.map { date => + val time = date.getTime() + if (finishTime > time) { + finishTime - time } else { None + currentTime - time } + } val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val inputRead = stageData.inputBytes val inputReadWithUnit = if (inputRead > 0) Utils.bytesToString(inputRead) else "" val outputWrite = stageData.outputBytes val outputWriteWithUnit = if (outputWrite > 0) Utils.bytesToString(outputWrite) else "" - val shuffleRead = stageData.shuffleReadTotalBytes + val shuffleRead = stageData.shuffleReadBytes val shuffleReadWithUnit = if (shuffleRead > 0) Utils.bytesToString(shuffleRead) else "" val shuffleWrite = stageData.shuffleWriteBytes val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else "" new StageTableRowData( - s, - stageDataOption, - s.stageId, - s.attemptId, + stageData, + Some(stageData), + stageData.stageId, + stageData.attemptId, stageData.schedulingPool, - description, - s.submissionTime.getOrElse(0), + stageData.description, + stageData.submissionTime.getOrElse(new Date(0)), formattedSubmissionTime, duration.getOrElse(-1), formattedDuration, @@ -496,7 +490,7 @@ private[ui] class StageDataSource( val ordering: Ordering[StageTableRowData] = sortColumn match { case "Stage Id" => Ordering.by(_.stageId) case "Pool Name" => Ordering.by(_.schedulingPool) - case "Description" => Ordering.by(x => (x.descriptionOption, x.stageInfo.name)) + case "Description" => Ordering.by(x => (x.descriptionOption, x.stage.name)) case "Submitted" => Ordering.by(_.submissionTime) case "Duration" => Ordering.by(_.duration) case "Input" => Ordering.by(_.inputRead) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 65446f967ad7..7631ee2d0cfe 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -17,40 +17,49 @@ package org.apache.spark.ui.jobs +import java.util.NoSuchElementException import javax.servlet.http.HttpServletRequest import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.api.v1.StageStatus import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils} /** Web UI showing progress status of all stages in the given SparkContext. */ -private[ui] class StagesTab(val parent: SparkUI, store: AppStatusStore) +private[ui] class StagesTab(val parent: SparkUI, val store: AppStatusStore) extends SparkUITab(parent, "stages") { val sc = parent.sc val conf = parent.conf val killEnabled = parent.killEnabled - val progressListener = parent.jobProgressListener - val operationGraphListener = parent.operationGraphListener - val lastUpdateTime = parent.lastUpdateTime attachPage(new AllStagesPage(this)) attachPage(new StagePage(this, store)) attachPage(new PoolPage(this)) - def isFairScheduler: Boolean = progressListener.schedulingMode == Some(SchedulingMode.FAIR) + def isFairScheduler: Boolean = { + store.environmentInfo().sparkProperties.toMap + .get("spark.scheduler.mode") + .map { mode => mode == SchedulingMode.FAIR } + .getOrElse(false) + } def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { // stripXSS is called first to remove suspicious characters used in XSS attacks val stageId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) stageId.foreach { id => - if (progressListener.activeStages.contains(id)) { - sc.foreach(_.cancelStage(id, "killed via the Web UI")) - // Do a quick pause here to give Spark time to kill the stage so it shows up as - // killed after the refresh. Note that this will block the serving thread so the - // time should be limited in duration. - Thread.sleep(100) + try { + val status = store.lastStageAttempt(id).status + if (status == StageStatus.ACTIVE || status == StageStatus.PENDING) { + sc.foreach(_.cancelStage(id, "killed via the Web UI")) + // Do a quick pause here to give Spark time to kill the stage so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } catch { + case _: NoSuchElementException => } } } diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index bb763248cd7e..827a8637b9bd 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -35,20 +35,20 @@ import org.apache.spark.storage.StorageLevel * nodes and children clusters. Additionally, a graph may also have edges that enter or exit * the graph from nodes that belong to adjacent graphs. */ -private[ui] case class RDDOperationGraph( +private[spark] case class RDDOperationGraph( edges: Seq[RDDOperationEdge], outgoingEdges: Seq[RDDOperationEdge], incomingEdges: Seq[RDDOperationEdge], rootCluster: RDDOperationCluster) /** A node in an RDDOperationGraph. This represents an RDD. */ -private[ui] case class RDDOperationNode(id: Int, name: String, cached: Boolean, callsite: String) +private[spark] case class RDDOperationNode(id: Int, name: String, cached: Boolean, callsite: String) /** * A directed edge connecting two nodes in an RDDOperationGraph. * This represents an RDD dependency. */ -private[ui] case class RDDOperationEdge(fromId: Int, toId: Int) +private[spark] case class RDDOperationEdge(fromId: Int, toId: Int) /** * A cluster that groups nodes together in an RDDOperationGraph. @@ -56,7 +56,7 @@ private[ui] case class RDDOperationEdge(fromId: Int, toId: Int) * This represents any grouping of RDDs, including operation scopes (e.g. textFile, flatMap), * stages, jobs, or any higher level construct. A cluster may be nested inside of other clusters. */ -private[ui] class RDDOperationCluster(val id: String, private var _name: String) { +private[spark] class RDDOperationCluster(val id: String, private var _name: String) { private val _childNodes = new ListBuffer[RDDOperationNode] private val _childClusters = new ListBuffer[RDDOperationCluster] @@ -92,7 +92,7 @@ private[ui] class RDDOperationCluster(val id: String, private var _name: String) } } -private[ui] object RDDOperationGraph extends Logging { +private[spark] object RDDOperationGraph extends Logging { val STAGE_CLUSTER_PREFIX = "stage_" diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala deleted file mode 100644 index 37a12a864693..000000000000 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala +++ /dev/null @@ -1,150 +0,0 @@ -/* - * 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.ui.scope - -import scala.collection.mutable - -import org.apache.spark.SparkConf -import org.apache.spark.scheduler._ -import org.apache.spark.ui.SparkUI - -/** - * A SparkListener that constructs a DAG of RDD operations. - */ -private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListener { - - // Note: the fate of jobs and stages are tied. This means when we clean up a job, - // we always clean up all of its stages. Similarly, when we clean up a stage, we - // always clean up its job (and, transitively, other stages in the same job). - private[ui] val jobIdToStageIds = new mutable.HashMap[Int, Seq[Int]] - private[ui] val jobIdToSkippedStageIds = new mutable.HashMap[Int, Seq[Int]] - private[ui] val stageIdToJobId = new mutable.HashMap[Int, Int] - private[ui] val stageIdToGraph = new mutable.HashMap[Int, RDDOperationGraph] - private[ui] val completedStageIds = new mutable.HashSet[Int] - - // Keep track of the order in which these are inserted so we can remove old ones - private[ui] val jobIds = new mutable.ArrayBuffer[Int] - private[ui] val stageIds = new mutable.ArrayBuffer[Int] - - // How many root nodes to retain in DAG Graph - private[ui] val retainedNodes = - conf.getInt("spark.ui.dagGraph.retainedRootRDDs", Int.MaxValue) - - // How many jobs or stages to retain graph metadata for - private val retainedJobs = - conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS) - private val retainedStages = - conf.getInt("spark.ui.retainedStages", SparkUI.DEFAULT_RETAINED_STAGES) - - /** - * Return the graph metadata for all stages in the given job. - * An empty list is returned if one or more of its stages has been cleaned up. - */ - def getOperationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = synchronized { - val skippedStageIds = jobIdToSkippedStageIds.getOrElse(jobId, Seq.empty) - val graphs = jobIdToStageIds.getOrElse(jobId, Seq.empty) - .flatMap { sid => stageIdToGraph.get(sid) } - // Mark any skipped stages as such - graphs.foreach { g => - val stageId = g.rootCluster.id.replaceAll(RDDOperationGraph.STAGE_CLUSTER_PREFIX, "").toInt - if (skippedStageIds.contains(stageId) && !g.rootCluster.name.contains("skipped")) { - g.rootCluster.setName(g.rootCluster.name + " (skipped)") - } - } - graphs - } - - /** Return the graph metadata for the given stage, or None if no such information exists. */ - def getOperationGraphForStage(stageId: Int): Option[RDDOperationGraph] = synchronized { - stageIdToGraph.get(stageId) - } - - /** On job start, construct a RDDOperationGraph for each stage in the job for display later. */ - override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized { - val jobId = jobStart.jobId - val stageInfos = jobStart.stageInfos - - jobIds += jobId - jobIdToStageIds(jobId) = jobStart.stageInfos.map(_.stageId).sorted - - stageInfos.foreach { stageInfo => - val stageId = stageInfo.stageId - stageIds += stageId - stageIdToJobId(stageId) = jobId - stageIdToGraph(stageId) = RDDOperationGraph.makeOperationGraph(stageInfo, retainedNodes) - trimStagesIfNecessary() - } - - trimJobsIfNecessary() - } - - /** Keep track of stages that have completed. */ - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { - val stageId = stageCompleted.stageInfo.stageId - if (stageIdToJobId.contains(stageId)) { - // Note: Only do this if the stage has not already been cleaned up - // Otherwise, we may never clean this stage from `completedStageIds` - completedStageIds += stageCompleted.stageInfo.stageId - } - } - - /** On job end, find all stages in this job that are skipped and mark them as such. */ - override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = synchronized { - val jobId = jobEnd.jobId - jobIdToStageIds.get(jobId).foreach { stageIds => - val skippedStageIds = stageIds.filter { sid => !completedStageIds.contains(sid) } - // Note: Only do this if the job has not already been cleaned up - // Otherwise, we may never clean this job from `jobIdToSkippedStageIds` - jobIdToSkippedStageIds(jobId) = skippedStageIds - } - } - - /** Clean metadata for old stages if we have exceeded the number to retain. */ - private def trimStagesIfNecessary(): Unit = { - if (stageIds.size >= retainedStages) { - val toRemove = math.max(retainedStages / 10, 1) - stageIds.take(toRemove).foreach { id => cleanStage(id) } - stageIds.trimStart(toRemove) - } - } - - /** Clean metadata for old jobs if we have exceeded the number to retain. */ - private def trimJobsIfNecessary(): Unit = { - if (jobIds.size >= retainedJobs) { - val toRemove = math.max(retainedJobs / 10, 1) - jobIds.take(toRemove).foreach { id => cleanJob(id) } - jobIds.trimStart(toRemove) - } - } - - /** Clean metadata for the given stage, its job, and all other stages that belong to the job. */ - private[ui] def cleanStage(stageId: Int): Unit = { - completedStageIds.remove(stageId) - stageIdToGraph.remove(stageId) - stageIdToJobId.remove(stageId).foreach { jobId => cleanJob(jobId) } - } - - /** Clean metadata for the given job and all stages that belong to it. */ - private[ui] def cleanJob(jobId: Int): Unit = { - jobIdToSkippedStageIds.remove(jobId) - jobIdToStageIds.remove(jobId).foreach { stageIds => - stageIds.foreach { stageId => cleanStage(stageId) } - } - } - -} diff --git a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json index 25c4fff77e0a..37b7d7269059 100644 --- a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json @@ -2,9 +2,12 @@ "status" : "COMPLETE", "stageId" : 3, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 162, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:07.191GMT", @@ -23,14 +26,19 @@ "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", - "accumulatorUpdates" : [ ] + "rddIds" : [ 6, 5 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } }, { "status" : "COMPLETE", "stageId" : 1, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 3476, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", @@ -49,14 +57,19 @@ "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" : [ ] + "rddIds" : [ 1, 0 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } }, { "status" : "COMPLETE", "stageId" : 0, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 4338, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:04.228GMT", @@ -75,5 +88,7 @@ "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", - "accumulatorUpdates" : [ ] + "rddIds" : [ 0 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json index b86ba1e65de1..2fd55666fa01 100644 --- a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json @@ -2,14 +2,18 @@ "status" : "FAILED", "stageId" : 2, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 7, "numFailedTasks" : 1, + "numKilledTasks" : 0, + "numCompletedIndices" : 7, "executorRunTime" : 278, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:06.296GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", "completionTime" : "2015-02-03T16:43:06.347GMT", + "failureReason" : "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(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(: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(Thread.java:745)\n\nDriver stacktrace:", "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, @@ -23,5 +27,7 @@ "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", - "accumulatorUpdates" : [ ] + "rddIds" : [ 3, 2 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json index c108fa61a431..2f275c7bfe2f 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_1__expectation.json @@ -8,8 +8,11 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json index c108fa61a431..2f275c7bfe2f 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_from_multi_attempt_app_json_2__expectation.json @@ -8,8 +8,11 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json index 3d7407004d26..71bf8706307c 100644 --- a/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/job_list_json_expectation.json @@ -8,10 +8,13 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } }, { "jobId" : 1, "name" : "count at :20", @@ -22,10 +25,13 @@ "numCompletedTasks" : 15, "numSkippedTasks" : 0, "numFailedTasks" : 1, + "numKilledTasks" : 0, + "numCompletedIndices" : 15, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 1 + "numFailedStages" : 1, + "killedTasksSummary" : { } }, { "jobId" : 0, "name" : "count at :15", @@ -36,8 +42,11 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json index 10c7e1c0b36f..1eae5f3d5beb 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_job_json_expectation.json @@ -8,8 +8,11 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 6fb40f6f1713..31093a661663 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -2,9 +2,12 @@ "status" : "COMPLETE", "stageId" : 1, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 3476, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", @@ -23,14 +26,15 @@ "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", + "rddIds" : [ 1, 0 ], "accumulatorUpdates" : [ ], "tasks" : { - "8" : { - "taskId" : 8, - "index" : 0, + "10" : { + "taskId" : 10, + "index" : 2, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.829GMT", - "duration" : 435, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 456, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -38,15 +42,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, + "executorDeserializeTime" : 2, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 435, + "executorRunTime" : 434, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -66,17 +71,17 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 94000, + "writeTime" : 76000, "recordsWritten" : 0 } } }, - "9" : { - "taskId" : 9, - "index" : 1, + "14" : { + "taskId" : 14, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "duration" : 436, + "launchTime" : "2015-02-03T16:43:05.832GMT", + "duration" : 450, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -84,15 +89,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, + "executorDeserializeTime" : 2, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 436, + "executorRunTime" : 434, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -112,17 +118,17 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 98000, + "writeTime" : 88000, "recordsWritten" : 0 } } }, - "10" : { - "taskId" : 10, - "index" : 2, + "9" : { + "taskId" : 9, + "index" : 1, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", - "duration" : 434, + "duration" : 454, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -130,15 +136,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 1, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 434, + "executorRunTime" : 436, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -158,17 +165,17 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 76000, + "writeTime" : 98000, "recordsWritten" : 0 } } }, - "11" : { - "taskId" : 11, - "index" : 3, + "13" : { + "taskId" : 13, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "duration" : 434, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 452, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -182,9 +189,10 @@ "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -203,8 +211,8 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, + "bytesWritten" : 1648, + "writeTime" : 73000, "recordsWritten" : 0 } } @@ -214,7 +222,7 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", - "duration" : 434, + "duration" : 454, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -231,6 +239,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -255,12 +264,12 @@ } } }, - "13" : { - "taskId" : 13, - "index" : 5, + "11" : { + "taskId" : 11, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", - "duration" : 434, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 454, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -274,9 +283,10 @@ "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -295,18 +305,18 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, + "bytesWritten" : 1647, + "writeTime" : 83000, "recordsWritten" : 0 } } }, - "14" : { - "taskId" : 14, - "index" : 6, + "8" : { + "taskId" : 8, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", - "duration" : 434, + "launchTime" : "2015-02-03T16:43:05.829GMT", + "duration" : 454, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -314,15 +324,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 1, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 434, + "executorRunTime" : 435, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -342,7 +353,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 88000, + "writeTime" : 94000, "recordsWritten" : 0 } } @@ -352,7 +363,7 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", - "duration" : 435, + "duration" : 450, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -369,6 +380,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -399,12 +411,18 @@ "taskTime" : 3624, "failedTasks" : 0, "succeededTasks" : 8, + "killedTasks" : 0, "inputBytes" : 28000128, + "inputRecords" : 0, "outputBytes" : 0, + "outputRecords" : 0, "shuffleRead" : 0, + "shuffleReadRecords" : 0, "shuffleWrite" : 13180, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0 } - } + }, + "killedTasksSummary" : { } } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index f5a89a210764..601d70695b17 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -2,9 +2,12 @@ "status" : "COMPLETE", "stageId" : 1, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 3476, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", @@ -23,14 +26,15 @@ "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", + "rddIds" : [ 1, 0 ], "accumulatorUpdates" : [ ], "tasks" : { - "8" : { - "taskId" : 8, - "index" : 0, + "10" : { + "taskId" : 10, + "index" : 2, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.829GMT", - "duration" : 435, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 456, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -38,15 +42,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, + "executorDeserializeTime" : 2, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 435, + "executorRunTime" : 434, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -66,17 +71,17 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 94000, + "writeTime" : 76000, "recordsWritten" : 0 } } }, - "9" : { - "taskId" : 9, - "index" : 1, + "14" : { + "taskId" : 14, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "duration" : 436, + "launchTime" : "2015-02-03T16:43:05.832GMT", + "duration" : 450, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -84,15 +89,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 1, + "executorDeserializeTime" : 2, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 436, + "executorRunTime" : 434, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -112,17 +118,17 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 98000, + "writeTime" : 88000, "recordsWritten" : 0 } } }, - "10" : { - "taskId" : 10, - "index" : 2, + "9" : { + "taskId" : 9, + "index" : 1, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", - "duration" : 434, + "duration" : 454, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -130,15 +136,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 1, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 434, + "executorRunTime" : 436, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -158,17 +165,17 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 76000, + "writeTime" : 98000, "recordsWritten" : 0 } } }, - "11" : { - "taskId" : 11, - "index" : 3, + "13" : { + "taskId" : 13, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.830GMT", - "duration" : 434, + "launchTime" : "2015-02-03T16:43:05.831GMT", + "duration" : 452, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -182,9 +189,10 @@ "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -203,8 +211,8 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1647, - "writeTime" : 83000, + "bytesWritten" : 1648, + "writeTime" : 73000, "recordsWritten" : 0 } } @@ -214,7 +222,7 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", - "duration" : 434, + "duration" : 454, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -231,6 +239,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -255,12 +264,12 @@ } } }, - "13" : { - "taskId" : 13, - "index" : 5, + "11" : { + "taskId" : 11, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.831GMT", - "duration" : 434, + "launchTime" : "2015-02-03T16:43:05.830GMT", + "duration" : 454, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -274,9 +283,10 @@ "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -295,18 +305,18 @@ "recordsRead" : 0 }, "shuffleWriteMetrics" : { - "bytesWritten" : 1648, - "writeTime" : 73000, + "bytesWritten" : 1647, + "writeTime" : 83000, "recordsWritten" : 0 } } }, - "14" : { - "taskId" : 14, - "index" : 6, + "8" : { + "taskId" : 8, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-02-03T16:43:05.832GMT", - "duration" : 434, + "launchTime" : "2015-02-03T16:43:05.829GMT", + "duration" : 454, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -314,15 +324,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 2, + "executorDeserializeTime" : 1, "executorDeserializeCpuTime" : 0, - "executorRunTime" : 434, + "executorRunTime" : 435, "executorCpuTime" : 0, "resultSize" : 1902, "jvmGcTime" : 19, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -342,7 +353,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, - "writeTime" : 88000, + "writeTime" : 94000, "recordsWritten" : 0 } } @@ -352,7 +363,7 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", - "duration" : 435, + "duration" : 450, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -369,6 +380,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 3500016, "recordsRead" : 0 @@ -399,12 +411,18 @@ "taskTime" : 3624, "failedTasks" : 0, "succeededTasks" : 8, + "killedTasks" : 0, "inputBytes" : 28000128, + "inputRecords" : 0, "outputBytes" : 0, + "outputRecords" : 0, "shuffleRead" : 0, + "shuffleReadRecords" : 0, "shuffleWrite" : 13180, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0 } - } + }, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json index 6509df1508b3..1e6fb40d6028 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json @@ -2,9 +2,12 @@ "status" : "COMPLETE", "stageId" : 3, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 162, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:07.191GMT", @@ -23,14 +26,51 @@ "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", - "accumulatorUpdates" : [ ] + "rddIds" : [ 6, 5 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } +}, { + "status" : "FAILED", + "stageId" : 2, + "attemptId" : 0, + "numTasks" : 8, + "numActiveTasks" : 0, + "numCompleteTasks" : 7, + "numFailedTasks" : 1, + "numKilledTasks" : 0, + "numCompletedIndices" : 7, + "executorRunTime" : 278, + "executorCpuTime" : 0, + "submissionTime" : "2015-02-03T16:43:06.296GMT", + "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", + "completionTime" : "2015-02-03T16:43:06.347GMT", + "failureReason" : "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(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(: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(Thread.java:745)\n\nDriver stacktrace:", + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "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", + "rddIds" : [ 3, 2 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } }, { "status" : "COMPLETE", "stageId" : 1, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 3476, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", @@ -49,14 +89,19 @@ "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" : [ ] + "rddIds" : [ 1, 0 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } }, { "status" : "COMPLETE", "stageId" : 0, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 4338, "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:04.228GMT", @@ -75,31 +120,7 @@ "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", - "accumulatorUpdates" : [ ] -}, { - "status" : "FAILED", - "stageId" : 2, - "attemptId" : 0, - "numActiveTasks" : 0, - "numCompleteTasks" : 7, - "numFailedTasks" : 1, - "executorRunTime" : 278, - "executorCpuTime" : 0, - "submissionTime" : "2015-02-03T16:43:06.296GMT", - "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", - "completionTime" : "2015-02-03T16:43:06.347GMT", - "inputBytes" : 0, - "inputRecords" : 0, - "outputBytes" : 0, - "outputRecords" : 0, - "shuffleReadBytes" : 0, - "shuffleReadRecords" : 0, - "shuffleWriteBytes" : 0, - "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "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", - "accumulatorUpdates" : [ ] + "rddIds" : [ 0 ], + "accumulatorUpdates" : [ ], + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json index 8496863a9346..e6284ccf9b73 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json @@ -2,9 +2,12 @@ "status" : "COMPLETE", "stageId" : 0, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 120, "executorCpuTime" : 0, "submissionTime" : "2015-03-16T19:25:36.103GMT", @@ -23,9 +26,11 @@ "name" : "foreach at :15", "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)", "schedulingPool" : "default", + "rddIds" : [ 0 ], "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", "value" : "5050" - } ] + } ], + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json index 9b401b414f8d..a15ee2352336 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -3,7 +3,7 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.494GMT", - "duration" : 349, + "duration" : 435, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -20,6 +20,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 49294, "recordsRead" : 10000 @@ -48,7 +49,7 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.502GMT", - "duration" : 350, + "duration" : 421, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -65,6 +66,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -93,7 +95,7 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", - "duration" : 348, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -110,6 +112,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -138,7 +141,7 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", - "duration" : 349, + "duration" : 423, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -155,6 +158,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -183,7 +187,7 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", - "duration" : 349, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -200,6 +204,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -228,7 +233,7 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", - "duration" : 350, + "duration" : 414, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -245,6 +250,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -273,7 +279,7 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", - "duration" : 351, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -290,6 +296,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -318,7 +325,7 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.506GMT", - "duration" : 349, + "duration" : 423, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -335,6 +342,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -363,7 +371,7 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", - "duration" : 80, + "duration" : 88, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -380,6 +388,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -408,7 +417,7 @@ "index" : 9, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.915GMT", - "duration" : 84, + "duration" : 101, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -425,6 +434,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60489, "recordsRead" : 10000 @@ -453,7 +463,7 @@ "index" : 10, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.916GMT", - "duration" : 73, + "duration" : 99, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -470,6 +480,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -498,7 +509,7 @@ "index" : 11, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.918GMT", - "duration" : 75, + "duration" : 89, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -515,6 +526,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -543,7 +555,7 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", - "duration" : 77, + "duration" : 93, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -560,6 +572,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -588,7 +601,7 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", - "duration" : 76, + "duration" : 138, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -605,6 +618,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -633,7 +647,7 @@ "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", - "duration" : 83, + "duration" : 94, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -650,6 +664,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -678,7 +693,7 @@ "index" : 15, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.928GMT", - "duration" : 76, + "duration" : 83, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -695,6 +710,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -723,7 +739,7 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", - "duration" : 84, + "duration" : 98, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -740,6 +756,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -768,7 +785,7 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", - "duration" : 91, + "duration" : 123, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -785,6 +802,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -813,7 +831,7 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", - "duration" : 92, + "duration" : 105, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -830,6 +848,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -858,7 +877,7 @@ "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", - "duration" : 84, + "duration" : 94, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -875,6 +894,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json index 2ebee66a6d7c..f9182b165833 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json @@ -3,7 +3,7 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.515GMT", - "duration" : 15, + "duration" : 61, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -25,6 +25,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -53,7 +54,7 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.521GMT", - "duration" : 15, + "duration" : 53, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -75,6 +76,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -103,7 +105,7 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 15, + "duration" : 48, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -125,6 +127,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -153,7 +156,7 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 15, + "duration" : 50, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -175,6 +178,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -203,7 +207,7 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 15, + "duration" : 52, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -225,6 +229,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -253,7 +258,7 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", - "duration" : 15, + "duration" : 52, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -275,6 +280,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -303,7 +309,7 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", - "duration" : 15, + "duration" : 51, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -325,6 +331,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -353,7 +360,7 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.524GMT", - "duration" : 15, + "duration" : 51, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -375,6 +382,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json index 965a31a4104c..76dd2f710b90 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json @@ -3,7 +3,7 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.515GMT", - "duration" : 15, + "duration" : 61, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -25,6 +25,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -53,7 +54,7 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.521GMT", - "duration" : 15, + "duration" : 53, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -75,6 +76,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -103,7 +105,7 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", - "duration" : 15, + "duration" : 48, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -125,6 +127,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -153,7 +156,7 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", - "duration" : 15, + "duration" : 50, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -175,6 +178,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -203,7 +207,7 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", - "duration" : 15, + "duration" : 52, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -225,6 +229,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -253,7 +258,7 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", - "duration" : 15, + "duration" : 52, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -275,6 +280,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -303,7 +309,7 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", - "duration" : 15, + "duration" : 51, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -325,6 +331,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -353,7 +360,7 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.524GMT", - "duration" : 15, + "duration" : 51, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -375,6 +382,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json index 31132e156937..6bdc10465d89 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json @@ -3,7 +3,7 @@ "index" : 10, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.916GMT", - "duration" : 73, + "duration" : 99, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -20,6 +20,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -48,7 +49,7 @@ "index" : 11, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.918GMT", - "duration" : 75, + "duration" : 89, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -65,6 +66,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -93,7 +95,7 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", - "duration" : 77, + "duration" : 93, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -110,6 +112,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -138,7 +141,7 @@ "index" : 13, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.924GMT", - "duration" : 76, + "duration" : 138, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -155,6 +158,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -183,7 +187,7 @@ "index" : 14, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.925GMT", - "duration" : 83, + "duration" : 94, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -200,6 +204,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -228,7 +233,7 @@ "index" : 15, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.928GMT", - "duration" : 76, + "duration" : 83, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -245,6 +250,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -273,7 +279,7 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", - "duration" : 84, + "duration" : 98, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -290,6 +296,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -318,7 +325,7 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", - "duration" : 91, + "duration" : 123, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -335,6 +342,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -363,7 +371,7 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", - "duration" : 92, + "duration" : 105, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -380,6 +388,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -408,7 +417,7 @@ "index" : 19, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.012GMT", - "duration" : 84, + "duration" : 94, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -425,6 +434,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -453,7 +463,7 @@ "index" : 20, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.014GMT", - "duration" : 83, + "duration" : 90, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -470,6 +480,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -498,7 +509,7 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", - "duration" : 88, + "duration" : 96, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -515,6 +526,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -543,7 +555,7 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", - "duration" : 93, + "duration" : 101, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -560,6 +572,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -588,7 +601,7 @@ "index" : 23, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.031GMT", - "duration" : 65, + "duration" : 84, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -605,6 +618,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -633,7 +647,7 @@ "index" : 24, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.098GMT", - "duration" : 43, + "duration" : 52, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -650,6 +664,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -678,7 +693,7 @@ "index" : 25, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.103GMT", - "duration" : 49, + "duration" : 61, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -695,6 +710,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -723,7 +739,7 @@ "index" : 26, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.105GMT", - "duration" : 38, + "duration" : 52, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -740,6 +756,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -768,7 +785,7 @@ "index" : 27, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.110GMT", - "duration" : 32, + "duration" : 41, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -785,6 +802,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -813,7 +831,7 @@ "index" : 28, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.113GMT", - "duration" : 29, + "duration" : 49, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -830,6 +848,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -858,7 +877,7 @@ "index" : 29, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.114GMT", - "duration" : 39, + "duration" : 52, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -875,6 +894,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -903,7 +923,7 @@ "index" : 30, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.118GMT", - "duration" : 34, + "duration" : 62, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -920,6 +940,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -948,7 +969,7 @@ "index" : 31, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.127GMT", - "duration" : 24, + "duration" : 74, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -965,6 +986,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -993,7 +1015,7 @@ "index" : 32, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.148GMT", - "duration" : 17, + "duration" : 33, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1010,6 +1032,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1038,7 +1061,7 @@ "index" : 33, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.149GMT", - "duration" : 43, + "duration" : 58, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1055,6 +1078,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1083,7 +1107,7 @@ "index" : 34, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.156GMT", - "duration" : 27, + "duration" : 42, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1100,6 +1124,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1128,7 +1153,7 @@ "index" : 35, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.161GMT", - "duration" : 35, + "duration" : 50, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1145,6 +1170,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1173,7 +1199,7 @@ "index" : 36, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.164GMT", - "duration" : 29, + "duration" : 40, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1190,6 +1216,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1218,7 +1245,7 @@ "index" : 37, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.165GMT", - "duration" : 32, + "duration" : 42, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1235,6 +1262,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1263,7 +1291,7 @@ "index" : 38, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.166GMT", - "duration" : 31, + "duration" : 47, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1280,6 +1308,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1308,7 +1337,7 @@ "index" : 39, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.180GMT", - "duration" : 17, + "duration" : 32, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1325,6 +1354,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1353,7 +1383,7 @@ "index" : 40, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.197GMT", - "duration" : 14, + "duration" : 24, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1370,6 +1400,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1398,7 +1429,7 @@ "index" : 41, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.200GMT", - "duration" : 16, + "duration" : 24, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1415,6 +1446,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1443,7 +1475,7 @@ "index" : 42, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.203GMT", - "duration" : 17, + "duration" : 42, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1460,6 +1492,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1488,7 +1521,7 @@ "index" : 43, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.204GMT", - "duration" : 16, + "duration" : 39, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1505,6 +1538,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1533,7 +1567,7 @@ "index" : 44, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.205GMT", - "duration" : 18, + "duration" : 37, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1550,6 +1584,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1578,7 +1613,7 @@ "index" : 45, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.206GMT", - "duration" : 19, + "duration" : 37, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1595,6 +1630,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1623,7 +1659,7 @@ "index" : 46, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.210GMT", - "duration" : 31, + "duration" : 43, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1640,6 +1676,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1668,7 +1705,7 @@ "index" : 47, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.212GMT", - "duration" : 18, + "duration" : 33, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1685,6 +1722,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1713,7 +1751,7 @@ "index" : 48, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.220GMT", - "duration" : 24, + "duration" : 30, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1730,6 +1768,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1758,7 +1797,7 @@ "index" : 49, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.223GMT", - "duration" : 23, + "duration" : 34, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1775,6 +1814,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1803,7 +1843,7 @@ "index" : 50, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.240GMT", - "duration" : 18, + "duration" : 26, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1820,6 +1860,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1848,7 +1889,7 @@ "index" : 51, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.242GMT", - "duration" : 17, + "duration" : 21, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1865,6 +1906,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1893,7 +1935,7 @@ "index" : 52, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.243GMT", - "duration" : 18, + "duration" : 28, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1910,6 +1952,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1938,7 +1981,7 @@ "index" : 53, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.244GMT", - "duration" : 18, + "duration" : 29, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -1955,6 +1998,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -1983,7 +2027,7 @@ "index" : 54, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.244GMT", - "duration" : 18, + "duration" : 59, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -2000,6 +2044,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2028,7 +2073,7 @@ "index" : 55, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.246GMT", - "duration" : 21, + "duration" : 30, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -2045,6 +2090,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2073,7 +2119,7 @@ "index" : 56, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.249GMT", - "duration" : 20, + "duration" : 31, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -2090,6 +2136,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2118,7 +2165,7 @@ "index" : 57, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.257GMT", - "duration" : 16, + "duration" : 21, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -2135,6 +2182,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2163,7 +2211,7 @@ "index" : 58, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.263GMT", - "duration" : 16, + "duration" : 23, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -2180,6 +2228,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -2208,7 +2257,7 @@ "index" : 59, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.265GMT", - "duration" : 17, + "duration" : 23, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -2225,6 +2274,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json index 6af1cfbeb8f7..bc1cd49909d3 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -3,7 +3,7 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", - "duration" : 351, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -20,6 +20,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -44,11 +45,11 @@ } } }, { - "taskId" : 1, - "index" : 1, + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.502GMT", - "duration" : 350, + "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 414, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -56,15 +57,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 30, "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -84,16 +86,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3934399, + "writeTime" : 3675510, "recordsWritten" : 10 } } }, { - "taskId" : 5, - "index" : 5, + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.505GMT", - "duration" : 350, + "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 421, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -101,15 +103,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 30, + "executorDeserializeTime" : 31, "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -129,16 +132,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3675510, + "writeTime" : 3934399, "recordsWritten" : 10 } } }, { - "taskId" : 0, - "index" : 0, + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.494GMT", - "duration" : 349, + "launchTime" : "2015-05-06T13:03:06.506GMT", + "duration" : 423, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -146,17 +149,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 32, + "executorDeserializeTime" : 31, "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 49294, + "bytesRead" : 60488, "recordsRead" : 10000 }, "outputMetrics" : { @@ -174,16 +178,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3842811, + "writeTime" : 2579051, "recordsWritten" : 10 } } }, { - "taskId" : 3, - "index" : 3, + "taskId" : 4, + "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", - "duration" : 349, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -197,9 +201,10 @@ "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -219,16 +224,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 1311694, + "writeTime" : 83022, "recordsWritten" : 10 } } }, { - "taskId" : 4, - "index" : 4, + "taskId" : 3, + "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", - "duration" : 349, + "duration" : 423, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -242,9 +247,10 @@ "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -264,16 +270,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 83022, + "writeTime" : 1311694, "recordsWritten" : 10 } } }, { - "taskId" : 7, - "index" : 7, + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.506GMT", - "duration" : 349, + "launchTime" : "2015-05-06T13:03:06.494GMT", + "duration" : 435, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -281,17 +287,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 32, "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 60488, + "bytesRead" : 49294, "recordsRead" : 10000 }, "outputMetrics" : { @@ -309,7 +316,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 2579051, + "writeTime" : 3842811, "recordsWritten" : 10 } } @@ -318,7 +325,7 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", - "duration" : 348, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -335,6 +342,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -363,7 +371,7 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", - "duration" : 93, + "duration" : 101, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -380,6 +388,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -408,7 +417,7 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", - "duration" : 92, + "duration" : 105, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -425,6 +434,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -453,7 +463,7 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", - "duration" : 91, + "duration" : 123, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -470,6 +480,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -498,7 +509,7 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", - "duration" : 88, + "duration" : 96, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -515,6 +526,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -539,11 +551,11 @@ } } }, { - "taskId" : 9, - "index" : 9, + "taskId" : 19, + "index" : 19, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.915GMT", - "duration" : 84, + "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 94, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -551,17 +563,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 5, "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 60489, + "bytesRead" : 70564, "recordsRead" : 10000 }, "outputMetrics" : { @@ -579,7 +592,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101664, + "writeTime" : 95788, "recordsWritten" : 10 } } @@ -588,7 +601,7 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", - "duration" : 84, + "duration" : 98, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -605,6 +618,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -629,11 +643,11 @@ } } }, { - "taskId" : 19, - "index" : 19, + "taskId" : 9, + "index" : 9, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.012GMT", - "duration" : 84, + "launchTime" : "2015-05-06T13:03:06.915GMT", + "duration" : 101, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -641,17 +655,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 5, + "executorDeserializeTime" : 9, "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 60489, "recordsRead" : 10000 }, "outputMetrics" : { @@ -669,16 +684,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95788, + "writeTime" : 101664, "recordsWritten" : 10 } } }, { - "taskId" : 14, - "index" : 14, + "taskId" : 20, + "index" : 20, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.925GMT", - "duration" : 83, + "launchTime" : "2015-05-06T13:03:07.014GMT", + "duration" : 90, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -686,15 +701,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 3, "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -714,16 +730,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95646, + "writeTime" : 97716, "recordsWritten" : 10 } } }, { - "taskId" : 20, - "index" : 20, + "taskId" : 14, + "index" : 14, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.014GMT", - "duration" : 83, + "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 94, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -731,15 +747,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 6, "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -759,7 +776,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 97716, + "writeTime" : 95646, "recordsWritten" : 10 } } @@ -768,7 +785,7 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", - "duration" : 80, + "duration" : 88, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -785,6 +802,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -813,7 +831,7 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", - "duration" : 77, + "duration" : 93, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -830,6 +848,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -854,11 +873,11 @@ } } }, { - "taskId" : 13, - "index" : 13, + "taskId" : 15, + "index" : 15, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.924GMT", - "duration" : 76, + "launchTime" : "2015-05-06T13:03:06.928GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -866,7 +885,7 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 3, "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, @@ -875,6 +894,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -894,7 +914,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95004, + "writeTime" : 602780, "recordsWritten" : 10 } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json index 6af1cfbeb8f7..bc1cd49909d3 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -3,7 +3,7 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.505GMT", - "duration" : 351, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -20,6 +20,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -44,11 +45,11 @@ } } }, { - "taskId" : 1, - "index" : 1, + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.502GMT", - "duration" : 350, + "launchTime" : "2015-05-06T13:03:06.505GMT", + "duration" : 414, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -56,15 +57,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 30, "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -84,16 +86,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3934399, + "writeTime" : 3675510, "recordsWritten" : 10 } } }, { - "taskId" : 5, - "index" : 5, + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.505GMT", - "duration" : 350, + "launchTime" : "2015-05-06T13:03:06.502GMT", + "duration" : 421, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -101,15 +103,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 30, + "executorDeserializeTime" : 31, "executorDeserializeCpuTime" : 0, "executorRunTime" : 350, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -129,16 +132,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3675510, + "writeTime" : 3934399, "recordsWritten" : 10 } } }, { - "taskId" : 0, - "index" : 0, + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.494GMT", - "duration" : 349, + "launchTime" : "2015-05-06T13:03:06.506GMT", + "duration" : 423, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -146,17 +149,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 32, + "executorDeserializeTime" : 31, "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 49294, + "bytesRead" : 60488, "recordsRead" : 10000 }, "outputMetrics" : { @@ -174,16 +178,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 3842811, + "writeTime" : 2579051, "recordsWritten" : 10 } } }, { - "taskId" : 3, - "index" : 3, + "taskId" : 4, + "index" : 4, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", - "duration" : 349, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -197,9 +201,10 @@ "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -219,16 +224,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 1311694, + "writeTime" : 83022, "recordsWritten" : 10 } } }, { - "taskId" : 4, - "index" : 4, + "taskId" : 3, + "index" : 3, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.504GMT", - "duration" : 349, + "duration" : 423, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -242,9 +247,10 @@ "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -264,16 +270,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 83022, + "writeTime" : 1311694, "recordsWritten" : 10 } } }, { - "taskId" : 7, - "index" : 7, + "taskId" : 0, + "index" : 0, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.506GMT", - "duration" : 349, + "launchTime" : "2015-05-06T13:03:06.494GMT", + "duration" : 435, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -281,17 +287,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 31, + "executorDeserializeTime" : 32, "executorDeserializeCpuTime" : 0, "executorRunTime" : 349, "executorCpuTime" : 0, "resultSize" : 2010, "jvmGcTime" : 7, - "resultSerializationTime" : 0, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 60488, + "bytesRead" : 49294, "recordsRead" : 10000 }, "outputMetrics" : { @@ -309,7 +316,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 2579051, + "writeTime" : 3842811, "recordsWritten" : 10 } } @@ -318,7 +325,7 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.503GMT", - "duration" : 348, + "duration" : 419, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -335,6 +342,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -363,7 +371,7 @@ "index" : 22, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.018GMT", - "duration" : 93, + "duration" : 101, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -380,6 +388,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -408,7 +417,7 @@ "index" : 18, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.010GMT", - "duration" : 92, + "duration" : 105, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -425,6 +434,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -453,7 +463,7 @@ "index" : 17, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.005GMT", - "duration" : 91, + "duration" : 123, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -470,6 +480,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -498,7 +509,7 @@ "index" : 21, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.015GMT", - "duration" : 88, + "duration" : 96, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -515,6 +526,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -539,11 +551,11 @@ } } }, { - "taskId" : 9, - "index" : 9, + "taskId" : 19, + "index" : 19, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.915GMT", - "duration" : 84, + "launchTime" : "2015-05-06T13:03:07.012GMT", + "duration" : 94, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -551,17 +563,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 5, "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 60489, + "bytesRead" : 70564, "recordsRead" : 10000 }, "outputMetrics" : { @@ -579,7 +592,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 101664, + "writeTime" : 95788, "recordsWritten" : 10 } } @@ -588,7 +601,7 @@ "index" : 16, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.001GMT", - "duration" : 84, + "duration" : 98, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -605,6 +618,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -629,11 +643,11 @@ } } }, { - "taskId" : 19, - "index" : 19, + "taskId" : 9, + "index" : 9, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.012GMT", - "duration" : 84, + "launchTime" : "2015-05-06T13:03:06.915GMT", + "duration" : 101, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -641,17 +655,18 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 5, + "executorDeserializeTime" : 9, "executorDeserializeCpuTime" : 0, "executorRunTime" : 84, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { - "bytesRead" : 70564, + "bytesRead" : 60489, "recordsRead" : 10000 }, "outputMetrics" : { @@ -669,16 +684,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95788, + "writeTime" : 101664, "recordsWritten" : 10 } } }, { - "taskId" : 14, - "index" : 14, + "taskId" : 20, + "index" : 20, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.925GMT", - "duration" : 83, + "launchTime" : "2015-05-06T13:03:07.014GMT", + "duration" : 90, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -686,15 +701,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 6, + "executorDeserializeTime" : 3, "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 0, + "jvmGcTime" : 5, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -714,16 +730,16 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95646, + "writeTime" : 97716, "recordsWritten" : 10 } } }, { - "taskId" : 20, - "index" : 20, + "taskId" : 14, + "index" : 14, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:07.014GMT", - "duration" : 83, + "launchTime" : "2015-05-06T13:03:06.925GMT", + "duration" : 94, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -731,15 +747,16 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 3, + "executorDeserializeTime" : 6, "executorDeserializeCpuTime" : 0, "executorRunTime" : 83, "executorCpuTime" : 0, "resultSize" : 2010, - "jvmGcTime" : 5, + "jvmGcTime" : 0, "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -759,7 +776,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 97716, + "writeTime" : 95646, "recordsWritten" : 10 } } @@ -768,7 +785,7 @@ "index" : 8, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.914GMT", - "duration" : 80, + "duration" : 88, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -785,6 +802,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 60488, "recordsRead" : 10000 @@ -813,7 +831,7 @@ "index" : 12, "attempt" : 0, "launchTime" : "2015-05-06T13:03:06.923GMT", - "duration" : 77, + "duration" : 93, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -830,6 +848,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -854,11 +873,11 @@ } } }, { - "taskId" : 13, - "index" : 13, + "taskId" : 15, + "index" : 15, "attempt" : 0, - "launchTime" : "2015-05-06T13:03:06.924GMT", - "duration" : 76, + "launchTime" : "2015-05-06T13:03:06.928GMT", + "duration" : 83, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -866,7 +885,7 @@ "speculative" : false, "accumulatorUpdates" : [ ], "taskMetrics" : { - "executorDeserializeTime" : 9, + "executorDeserializeTime" : 3, "executorDeserializeCpuTime" : 0, "executorRunTime" : 76, "executorCpuTime" : 0, @@ -875,6 +894,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -894,7 +914,7 @@ }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, - "writeTime" : 95004, + "writeTime" : 602780, "recordsWritten" : 10 } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json index c26daf4b8d7b..09857cb401ac 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -3,7 +3,7 @@ "index" : 40, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.197GMT", - "duration" : 14, + "duration" : 24, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -20,6 +20,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -48,7 +49,7 @@ "index" : 41, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.200GMT", - "duration" : 16, + "duration" : 24, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -65,6 +66,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -93,7 +95,7 @@ "index" : 43, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.204GMT", - "duration" : 16, + "duration" : 39, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -110,6 +112,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -138,7 +141,7 @@ "index" : 57, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.257GMT", - "duration" : 16, + "duration" : 21, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -155,6 +158,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -183,7 +187,7 @@ "index" : 58, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.263GMT", - "duration" : 16, + "duration" : 23, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -200,6 +204,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -228,7 +233,7 @@ "index" : 68, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.306GMT", - "duration" : 16, + "duration" : 22, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -245,6 +250,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -273,7 +279,7 @@ "index" : 86, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.374GMT", - "duration" : 16, + "duration" : 28, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -290,6 +296,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -318,7 +325,7 @@ "index" : 32, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.148GMT", - "duration" : 17, + "duration" : 33, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -335,6 +342,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -363,7 +371,7 @@ "index" : 39, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.180GMT", - "duration" : 17, + "duration" : 32, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -380,6 +388,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -408,7 +417,7 @@ "index" : 42, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.203GMT", - "duration" : 17, + "duration" : 42, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -425,6 +434,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -453,7 +463,7 @@ "index" : 51, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.242GMT", - "duration" : 17, + "duration" : 21, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -470,6 +480,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -498,7 +509,7 @@ "index" : 59, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.265GMT", - "duration" : 17, + "duration" : 23, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -515,6 +526,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -543,7 +555,7 @@ "index" : 63, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.276GMT", - "duration" : 17, + "duration" : 40, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -560,6 +572,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -588,7 +601,7 @@ "index" : 87, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.374GMT", - "duration" : 17, + "duration" : 36, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -605,6 +618,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -633,7 +647,7 @@ "index" : 90, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.385GMT", - "duration" : 17, + "duration" : 23, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -650,6 +664,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -678,7 +693,7 @@ "index" : 99, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.426GMT", - "duration" : 17, + "duration" : 22, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -695,6 +710,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70565, "recordsRead" : 10000 @@ -723,7 +739,7 @@ "index" : 44, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.205GMT", - "duration" : 18, + "duration" : 37, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -740,6 +756,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -768,7 +785,7 @@ "index" : 47, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.212GMT", - "duration" : 18, + "duration" : 33, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -785,6 +802,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -813,7 +831,7 @@ "index" : 50, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.240GMT", - "duration" : 18, + "duration" : 26, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -830,6 +848,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 @@ -858,7 +877,7 @@ "index" : 52, "attempt" : 0, "launchTime" : "2015-05-06T13:03:07.243GMT", - "duration" : 18, + "duration" : 28, "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", @@ -875,6 +894,7 @@ "resultSerializationTime" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 70564, "recordsRead" : 10000 diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index 44b5f66efe33..9cdcef074618 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -2,9 +2,12 @@ "status" : "COMPLETE", "stageId" : 0, "attemptId" : 0, + "numTasks" : 8, "numActiveTasks" : 0, "numCompleteTasks" : 8, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "executorRunTime" : 120, "executorCpuTime" : 0, "submissionTime" : "2015-03-16T19:25:36.103GMT", @@ -23,6 +26,7 @@ "name" : "foreach at :15", "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)", "schedulingPool" : "default", + "rddIds" : [ 0 ], "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", @@ -34,7 +38,7 @@ "index" : 0, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.515GMT", - "duration" : 15, + "duration" : 61, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -56,6 +60,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -85,7 +90,7 @@ "index" : 1, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.521GMT", - "duration" : 15, + "duration" : 53, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -107,6 +112,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -136,7 +142,7 @@ "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 15, + "duration" : 48, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -158,6 +164,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -187,7 +194,7 @@ "index" : 3, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 15, + "duration" : 50, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -209,6 +216,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -238,7 +246,7 @@ "index" : 4, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 15, + "duration" : 52, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -260,6 +268,7 @@ "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -289,7 +298,7 @@ "index" : 5, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", - "duration" : 15, + "duration" : 52, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -311,6 +320,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -340,7 +350,7 @@ "index" : 6, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", - "duration" : 15, + "duration" : 51, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -362,6 +372,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -391,7 +402,7 @@ "index" : 7, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.524GMT", - "duration" : 15, + "duration" : 51, "executorId" : "", "host" : "localhost", "status" : "SUCCESS", @@ -413,6 +424,7 @@ "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputMetrics" : { "bytesRead" : 0, "recordsRead" : 0 @@ -443,12 +455,18 @@ "taskTime" : 418, "failedTasks" : 0, "succeededTasks" : 8, + "killedTasks" : 0, "inputBytes" : 0, + "inputRecords" : 0, "outputBytes" : 0, + "outputRecords" : 0, "shuffleRead" : 0, + "shuffleReadRecords" : 0, "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0 } - } + }, + "killedTasksSummary" : { } } diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json index 3d7407004d26..71bf8706307c 100644 --- a/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_failed_job_list_json_expectation.json @@ -8,10 +8,13 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } }, { "jobId" : 1, "name" : "count at :20", @@ -22,10 +25,13 @@ "numCompletedTasks" : 15, "numSkippedTasks" : 0, "numFailedTasks" : 1, + "numKilledTasks" : 0, + "numCompletedIndices" : 15, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 1 + "numFailedStages" : 1, + "killedTasksSummary" : { } }, { "jobId" : 0, "name" : "count at :15", @@ -36,8 +42,11 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json index 6a9bafd6b219..b1ddd760c971 100644 --- a/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/succeeded_job_list_json_expectation.json @@ -8,10 +8,13 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } }, { "jobId" : 0, "name" : "count at :15", @@ -22,8 +25,11 @@ "numCompletedTasks" : 8, "numSkippedTasks" : 0, "numFailedTasks" : 0, + "numKilledTasks" : 0, + "numCompletedIndices" : 8, "numActiveStages" : 0, "numCompletedStages" : 1, "numSkippedStages" : 0, - "numFailedStages" : 0 + "numFailedStages" : 0, + "killedTasksSummary" : { } } ] 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 6a1abceaeb63..d22a19e8af74 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 @@ -23,6 +23,7 @@ import java.util.zip.ZipInputStream import javax.servlet._ import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpServletResponse} +import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.language.postfixOps @@ -44,8 +45,8 @@ import org.scalatest.selenium.WebBrowser import org.apache.spark._ import org.apache.spark.deploy.history.config._ +import org.apache.spark.status.api.v1.JobData import org.apache.spark.ui.SparkUI -import org.apache.spark.ui.jobs.UIData.JobUIData import org.apache.spark.util.{ResetSystemProperties, Utils} /** @@ -262,7 +263,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers 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]")) + badStageAttemptId._3 should be (Some("unknown attempt 1 for stage 1.")) val badStageId2 = getContentAndCode("applications/local-1422981780767/stages/flimflam") badStageId2._1 should be (HttpServletResponse.SC_NOT_FOUND) @@ -496,12 +497,12 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } - def completedJobs(): Seq[JobUIData] = { - getAppUI.jobProgressListener.completedJobs + def completedJobs(): Seq[JobData] = { + getAppUI.store.jobsList(List(JobExecutionStatus.SUCCEEDED).asJava) } - def activeJobs(): Seq[JobUIData] = { - getAppUI.jobProgressListener.activeJobs.values.toSeq + def activeJobs(): Seq[JobData] = { + getAppUI.store.jobsList(List(JobExecutionStatus.RUNNING).asJava) } activeJobs() should have size 0 diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 867d35f231dc..eabd4a678bc4 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -180,7 +180,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { check[StageDataWrapper](key(stages.head)) { stage => assert(stage.info.status === v1.StageStatus.ACTIVE) assert(stage.info.submissionTime === Some(new Date(stages.head.submissionTime.get))) - assert(stage.info.schedulingPool === "schedPool") + assert(stage.info.numTasks === stages.head.numTasks) } // Start tasks from stage 1 @@ -265,12 +265,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { "taskType", TaskResultLost, s1Tasks.head, null)) time += 1 - val reattempt = { - val orig = s1Tasks.head - // Task reattempts have a different ID, but the same index as the original. - new TaskInfo(nextTaskId(), orig.index, orig.attemptNumber + 1, time, orig.executorId, - s"${orig.executorId}.example.com", TaskLocality.PROCESS_LOCAL, orig.speculative) - } + val reattempt = newAttempt(s1Tasks.head, nextTaskId()) listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, reattempt)) @@ -288,7 +283,6 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { check[TaskDataWrapper](s1Tasks.head.taskId) { task => assert(task.info.status === s1Tasks.head.status) - assert(task.info.duration === Some(s1Tasks.head.duration)) assert(task.info.errorMessage == Some(TaskResultLost.toErrorString)) } @@ -297,8 +291,64 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(task.info.attempt === reattempt.attemptNumber) } + // Kill one task, restart it. + time += 1 + val killed = s1Tasks.drop(1).head + killed.finishTime = time + killed.failed = true + listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId, + "taskType", TaskKilled("killed"), killed, null)) + + check[JobDataWrapper](1) { job => + assert(job.info.numKilledTasks === 1) + assert(job.info.killedTasksSummary === Map("killed" -> 1)) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.numKilledTasks === 1) + assert(stage.info.killedTasksSummary === Map("killed" -> 1)) + } + + check[TaskDataWrapper](killed.taskId) { task => + assert(task.info.index === killed.index) + assert(task.info.errorMessage === Some("killed")) + } + + // Start a new attempt and finish it with TaskCommitDenied, make sure it's handled like a kill. + time += 1 + val denied = newAttempt(killed, nextTaskId()) + val denyReason = TaskCommitDenied(1, 1, 1) + listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, + denied)) + + time += 1 + denied.finishTime = time + denied.failed = true + listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId, + "taskType", denyReason, denied, null)) + + check[JobDataWrapper](1) { job => + assert(job.info.numKilledTasks === 2) + assert(job.info.killedTasksSummary === Map("killed" -> 1, denyReason.toErrorString -> 1)) + } + + check[StageDataWrapper](key(stages.head)) { stage => + assert(stage.info.numKilledTasks === 2) + assert(stage.info.killedTasksSummary === Map("killed" -> 1, denyReason.toErrorString -> 1)) + } + + check[TaskDataWrapper](denied.taskId) { task => + assert(task.info.index === killed.index) + assert(task.info.errorMessage === Some(denyReason.toErrorString)) + } + + // Start a new attempt. + val reattempt2 = newAttempt(denied, nextTaskId()) + listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, + reattempt2)) + // Succeed all tasks in stage 1. - val pending = s1Tasks.drop(1) ++ Seq(reattempt) + val pending = s1Tasks.drop(2) ++ Seq(reattempt, reattempt2) val s1Metrics = TaskMetrics.empty s1Metrics.setExecutorCpuTime(2L) @@ -313,12 +363,14 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { check[JobDataWrapper](1) { job => assert(job.info.numFailedTasks === 1) + assert(job.info.numKilledTasks === 2) assert(job.info.numActiveTasks === 0) assert(job.info.numCompletedTasks === pending.size) } check[StageDataWrapper](key(stages.head)) { stage => assert(stage.info.numFailedTasks === 1) + assert(stage.info.numKilledTasks === 2) assert(stage.info.numActiveTasks === 0) assert(stage.info.numCompleteTasks === pending.size) } @@ -328,10 +380,11 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(wrapper.info.errorMessage === None) assert(wrapper.info.taskMetrics.get.executorCpuTime === 2L) assert(wrapper.info.taskMetrics.get.executorRunTime === 4L) + assert(wrapper.info.duration === Some(task.duration)) } } - assert(store.count(classOf[TaskDataWrapper]) === pending.size + 1) + assert(store.count(classOf[TaskDataWrapper]) === pending.size + 3) // End stage 1. time += 1 @@ -404,6 +457,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(stage.info.numFailedTasks === s2Tasks.size) assert(stage.info.numActiveTasks === 0) assert(stage.info.numCompleteTasks === 0) + assert(stage.info.failureReason === stages.last.failureReason) } // - Re-submit stage 2, all tasks, and succeed them and the stage. @@ -740,4 +794,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { fn(value) } + private def newAttempt(orig: TaskInfo, nextId: Long): TaskInfo = { + // Task reattempts have a different ID, but the same index as the original. + new TaskInfo(nextId, orig.index, orig.attemptNumber + 1, time, orig.executorId, + s"${orig.executorId}.example.com", TaskLocality.PROCESS_LOCAL, orig.speculative) + } + } diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala deleted file mode 100644 index 82bd7c4ff660..000000000000 --- a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.mutable.LinkedHashMap - -import org.apache.spark.SparkFunSuite -import org.apache.spark.scheduler.{StageInfo, TaskInfo, TaskLocality} -import org.apache.spark.ui.jobs.UIData.{StageUIData, TaskUIData} - -class AllStagesResourceSuite extends SparkFunSuite { - - def getFirstTaskLaunchTime(taskLaunchTimes: Seq[Long]): Option[Date] = { - val tasks = new LinkedHashMap[Long, TaskUIData] - taskLaunchTimes.zipWithIndex.foreach { case (time, idx) => - tasks(idx.toLong) = TaskUIData( - new TaskInfo(idx, idx, 1, time, "", "", TaskLocality.ANY, false)) - } - - val stageUiData = new StageUIData() - stageUiData.taskData = tasks - val status = StageStatus.ACTIVE - val stageInfo = new StageInfo( - 1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc") - val stageData = AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, false) - - stageData.firstTaskLaunchedTime - } - - test("firstTaskLaunchedTime when there are no tasks") { - val result = getFirstTaskLaunchTime(Seq()) - assert(result == None) - } - - test("firstTaskLaunchedTime when there are tasks but none launched") { - val result = getFirstTaskLaunchTime(Seq(-100L, -200L, -300L)) - assert(result == None) - } - - test("firstTaskLaunchedTime when there are tasks and some launched") { - val result = getFirstTaskLaunchTime(Seq(-100L, 1449255596000L, 1449255597000L)) - assert(result == Some(new Date(1449255596000L))) - } - -} diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 1c51c148ae61..46932a02f1a1 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -29,8 +29,8 @@ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore -import org.apache.spark.ui.jobs.{JobProgressListener, StagePage, StagesTab} -import org.apache.spark.ui.scope.RDDOperationGraphListener +import org.apache.spark.ui.jobs.{StagePage, StagesTab} +import org.apache.spark.util.Utils class StagePageSuite extends SparkFunSuite with LocalSparkContext { @@ -55,38 +55,40 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { * This also runs a dummy stage to populate the page with useful content. */ private def renderStagePage(conf: SparkConf): Seq[Node] = { - val store = mock(classOf[AppStatusStore]) - when(store.executorSummary(anyString())).thenReturn(None) + val bus = new ReplayListenerBus() + val store = AppStatusStore.createLiveStore(conf, l => bus.addListener(l)) - val jobListener = new JobProgressListener(conf) - val graphListener = new RDDOperationGraphListener(conf) - val tab = mock(classOf[StagesTab], RETURNS_SMART_NULLS) - val request = mock(classOf[HttpServletRequest]) - when(tab.conf).thenReturn(conf) - when(tab.progressListener).thenReturn(jobListener) - when(tab.operationGraphListener).thenReturn(graphListener) - when(tab.appName).thenReturn("testing") - when(tab.headerTabs).thenReturn(Seq.empty) - when(request.getParameter("id")).thenReturn("0") - when(request.getParameter("attempt")).thenReturn("0") - val page = new StagePage(tab, store) + try { + val tab = mock(classOf[StagesTab], RETURNS_SMART_NULLS) + when(tab.store).thenReturn(store) - // Simulate a stage in job progress listener - val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") - // Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness - (1 to 2).foreach { - taskId => - val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, false) - jobListener.onStageSubmitted(SparkListenerStageSubmitted(stageInfo)) - jobListener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo)) - taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis()) - val taskMetrics = TaskMetrics.empty - taskMetrics.incPeakExecutionMemory(peakExecutionMemory) - jobListener.onTaskEnd( - SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) + val request = mock(classOf[HttpServletRequest]) + when(tab.conf).thenReturn(conf) + when(tab.appName).thenReturn("testing") + when(tab.headerTabs).thenReturn(Seq.empty) + when(request.getParameter("id")).thenReturn("0") + when(request.getParameter("attempt")).thenReturn("0") + val page = new StagePage(tab, store) + + // Simulate a stage in job progress listener + val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") + // Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness + (1 to 2).foreach { + taskId => + val taskInfo = new TaskInfo(taskId, taskId, 0, 0, "0", "localhost", TaskLocality.ANY, + false) + bus.postToAll(SparkListenerStageSubmitted(stageInfo)) + bus.postToAll(SparkListenerTaskStart(0, 0, taskInfo)) + taskInfo.markFinished(TaskState.FINISHED, System.currentTimeMillis()) + val taskMetrics = TaskMetrics.empty + taskMetrics.incPeakExecutionMemory(peakExecutionMemory) + bus.postToAll(SparkListenerTaskEnd(0, 0, "result", Success, taskInfo, taskMetrics)) + } + bus.postToAll(SparkListenerStageCompleted(stageInfo)) + page.render(request) + } finally { + store.close() } - jobListener.onStageCompleted(SparkListenerStageCompleted(stageInfo)) - page.render(request) } } 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 267c8dc1bd75..6a6c37873e1c 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -524,7 +524,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } } - test("stage & job retention") { + ignore("stage & job retention") { val conf = new SparkConf() .setMaster("local") .setAppName("test") @@ -670,34 +670,36 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) rdd.count() - val stage0 = Source.fromURL(sc.ui.get.webUrl + - "/stages/stage/?id=0&attempt=0&expandDagViz=true").mkString - assert(stage0.contains("digraph G {\n subgraph clusterstage_0 {\n " + - "label="Stage 0";\n subgraph ")) - assert(stage0.contains("{\n label="parallelize";\n " + - "0 [label="ParallelCollectionRDD [0]")) - assert(stage0.contains("{\n label="map";\n " + - "1 [label="MapPartitionsRDD [1]")) - assert(stage0.contains("{\n label="groupBy";\n " + - "2 [label="MapPartitionsRDD [2]")) - - val stage1 = Source.fromURL(sc.ui.get.webUrl + - "/stages/stage/?id=1&attempt=0&expandDagViz=true").mkString - assert(stage1.contains("digraph G {\n subgraph clusterstage_1 {\n " + - "label="Stage 1";\n subgraph ")) - assert(stage1.contains("{\n label="groupBy";\n " + - "3 [label="ShuffledRDD [3]")) - assert(stage1.contains("{\n label="map";\n " + - "4 [label="MapPartitionsRDD [4]")) - assert(stage1.contains("{\n label="groupBy";\n " + - "5 [label="MapPartitionsRDD [5]")) - - val stage2 = Source.fromURL(sc.ui.get.webUrl + - "/stages/stage/?id=2&attempt=0&expandDagViz=true").mkString - assert(stage2.contains("digraph G {\n subgraph clusterstage_2 {\n " + - "label="Stage 2";\n subgraph ")) - assert(stage2.contains("{\n label="groupBy";\n " + - "6 [label="ShuffledRDD [6]")) + eventually(timeout(5 seconds), interval(100 milliseconds)) { + val stage0 = Source.fromURL(sc.ui.get.webUrl + + "/stages/stage/?id=0&attempt=0&expandDagViz=true").mkString + assert(stage0.contains("digraph G {\n subgraph clusterstage_0 {\n " + + "label="Stage 0";\n subgraph ")) + assert(stage0.contains("{\n label="parallelize";\n " + + "0 [label="ParallelCollectionRDD [0]")) + assert(stage0.contains("{\n label="map";\n " + + "1 [label="MapPartitionsRDD [1]")) + assert(stage0.contains("{\n label="groupBy";\n " + + "2 [label="MapPartitionsRDD [2]")) + + val stage1 = Source.fromURL(sc.ui.get.webUrl + + "/stages/stage/?id=1&attempt=0&expandDagViz=true").mkString + assert(stage1.contains("digraph G {\n subgraph clusterstage_1 {\n " + + "label="Stage 1";\n subgraph ")) + assert(stage1.contains("{\n label="groupBy";\n " + + "3 [label="ShuffledRDD [3]")) + assert(stage1.contains("{\n label="map";\n " + + "4 [label="MapPartitionsRDD [4]")) + assert(stage1.contains("{\n label="groupBy";\n " + + "5 [label="MapPartitionsRDD [5]")) + + val stage2 = Source.fromURL(sc.ui.get.webUrl + + "/stages/stage/?id=2&attempt=0&expandDagViz=true").mkString + assert(stage2.contains("digraph G {\n subgraph clusterstage_2 {\n " + + "label="Stage 2";\n subgraph ")) + assert(stage2.contains("{\n label="groupBy";\n " + + "6 [label="ShuffledRDD [6]")) + } } } diff --git a/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphListenerSuite.scala deleted file mode 100644 index 3fb78da0c747..000000000000 --- a/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphListenerSuite.scala +++ /dev/null @@ -1,226 +0,0 @@ -/* - * 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.ui.scope - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.scheduler._ - -/** - * Tests that this listener populates and cleans up its data structures properly. - */ -class RDDOperationGraphListenerSuite extends SparkFunSuite { - private var jobIdCounter = 0 - private var stageIdCounter = 0 - private val maxRetainedJobs = 10 - private val maxRetainedStages = 10 - private val conf = new SparkConf() - .set("spark.ui.retainedJobs", maxRetainedJobs.toString) - .set("spark.ui.retainedStages", maxRetainedStages.toString) - - test("run normal jobs") { - val startingJobId = jobIdCounter - val startingStageId = stageIdCounter - val listener = new RDDOperationGraphListener(conf) - assert(listener.jobIdToStageIds.isEmpty) - assert(listener.jobIdToSkippedStageIds.isEmpty) - assert(listener.stageIdToJobId.isEmpty) - assert(listener.stageIdToGraph.isEmpty) - assert(listener.completedStageIds.isEmpty) - assert(listener.jobIds.isEmpty) - assert(listener.stageIds.isEmpty) - - // Run a few jobs, but not enough for clean up yet - (1 to 3).foreach { numStages => startJob(numStages, listener) } // start 3 jobs and 6 stages - (0 to 5).foreach { i => endStage(startingStageId + i, listener) } // finish all 6 stages - (0 to 2).foreach { i => endJob(startingJobId + i, listener) } // finish all 3 jobs - - assert(listener.jobIdToStageIds.size === 3) - assert(listener.jobIdToStageIds(startingJobId).size === 1) - assert(listener.jobIdToStageIds(startingJobId + 1).size === 2) - assert(listener.jobIdToStageIds(startingJobId + 2).size === 3) - assert(listener.jobIdToSkippedStageIds.size === 3) - assert(listener.jobIdToSkippedStageIds.values.forall(_.isEmpty)) // no skipped stages - assert(listener.stageIdToJobId.size === 6) - assert(listener.stageIdToJobId(startingStageId) === startingJobId) - assert(listener.stageIdToJobId(startingStageId + 1) === startingJobId + 1) - assert(listener.stageIdToJobId(startingStageId + 2) === startingJobId + 1) - assert(listener.stageIdToJobId(startingStageId + 3) === startingJobId + 2) - assert(listener.stageIdToJobId(startingStageId + 4) === startingJobId + 2) - assert(listener.stageIdToJobId(startingStageId + 5) === startingJobId + 2) - assert(listener.stageIdToGraph.size === 6) - assert(listener.completedStageIds.size === 6) - assert(listener.jobIds.size === 3) - assert(listener.stageIds.size === 6) - } - - test("run jobs with skipped stages") { - val startingJobId = jobIdCounter - val startingStageId = stageIdCounter - val listener = new RDDOperationGraphListener(conf) - - // Run a few jobs, but not enough for clean up yet - // Leave some stages unfinished so that they are marked as skipped - (1 to 3).foreach { numStages => startJob(numStages, listener) } // start 3 jobs and 6 stages - (4 to 5).foreach { i => endStage(startingStageId + i, listener) } // finish only last 2 stages - (0 to 2).foreach { i => endJob(startingJobId + i, listener) } // finish all 3 jobs - - assert(listener.jobIdToSkippedStageIds.size === 3) - assert(listener.jobIdToSkippedStageIds(startingJobId).size === 1) - assert(listener.jobIdToSkippedStageIds(startingJobId + 1).size === 2) - assert(listener.jobIdToSkippedStageIds(startingJobId + 2).size === 1) // 2 stages not skipped - assert(listener.completedStageIds.size === 2) - - // The rest should be the same as before - assert(listener.jobIdToStageIds.size === 3) - assert(listener.jobIdToStageIds(startingJobId).size === 1) - assert(listener.jobIdToStageIds(startingJobId + 1).size === 2) - assert(listener.jobIdToStageIds(startingJobId + 2).size === 3) - assert(listener.stageIdToJobId.size === 6) - assert(listener.stageIdToJobId(startingStageId) === startingJobId) - assert(listener.stageIdToJobId(startingStageId + 1) === startingJobId + 1) - assert(listener.stageIdToJobId(startingStageId + 2) === startingJobId + 1) - assert(listener.stageIdToJobId(startingStageId + 3) === startingJobId + 2) - assert(listener.stageIdToJobId(startingStageId + 4) === startingJobId + 2) - assert(listener.stageIdToJobId(startingStageId + 5) === startingJobId + 2) - assert(listener.stageIdToGraph.size === 6) - assert(listener.jobIds.size === 3) - assert(listener.stageIds.size === 6) - } - - test("clean up metadata") { - val startingJobId = jobIdCounter - val startingStageId = stageIdCounter - val listener = new RDDOperationGraphListener(conf) - - // Run many jobs and stages to trigger clean up - (1 to 10000).foreach { i => - // Note: this must be less than `maxRetainedStages` - val numStages = i % (maxRetainedStages - 2) + 1 - val startingStageIdForJob = stageIdCounter - val jobId = startJob(numStages, listener) - // End some, but not all, stages that belong to this job - // This is to ensure that we have both completed and skipped stages - (startingStageIdForJob until stageIdCounter) - .filter { i => i % 2 == 0 } - .foreach { i => endStage(i, listener) } - // End all jobs - endJob(jobId, listener) - } - - // Ensure we never exceed the max retained thresholds - assert(listener.jobIdToStageIds.size <= maxRetainedJobs) - assert(listener.jobIdToSkippedStageIds.size <= maxRetainedJobs) - assert(listener.stageIdToJobId.size <= maxRetainedStages) - assert(listener.stageIdToGraph.size <= maxRetainedStages) - assert(listener.completedStageIds.size <= maxRetainedStages) - assert(listener.jobIds.size <= maxRetainedJobs) - assert(listener.stageIds.size <= maxRetainedStages) - - // Also ensure we're actually populating these data structures - // Otherwise the previous group of asserts will be meaningless - assert(listener.jobIdToStageIds.nonEmpty) - assert(listener.jobIdToSkippedStageIds.nonEmpty) - assert(listener.stageIdToJobId.nonEmpty) - assert(listener.stageIdToGraph.nonEmpty) - assert(listener.completedStageIds.nonEmpty) - assert(listener.jobIds.nonEmpty) - assert(listener.stageIds.nonEmpty) - - // Ensure we clean up old jobs and stages, not arbitrary ones - assert(!listener.jobIdToStageIds.contains(startingJobId)) - assert(!listener.jobIdToSkippedStageIds.contains(startingJobId)) - assert(!listener.stageIdToJobId.contains(startingStageId)) - assert(!listener.stageIdToGraph.contains(startingStageId)) - assert(!listener.completedStageIds.contains(startingStageId)) - assert(!listener.stageIds.contains(startingStageId)) - assert(!listener.jobIds.contains(startingJobId)) - } - - test("fate sharing between jobs and stages") { - val startingJobId = jobIdCounter - val startingStageId = stageIdCounter - val listener = new RDDOperationGraphListener(conf) - - // Run 3 jobs and 8 stages, finishing all 3 jobs but only 2 stages - startJob(5, listener) - startJob(1, listener) - startJob(2, listener) - (0 until 8).foreach { i => startStage(i + startingStageId, listener) } - endStage(startingStageId + 3, listener) - endStage(startingStageId + 4, listener) - (0 until 3).foreach { i => endJob(i + startingJobId, listener) } - - // First, assert the old stuff - assert(listener.jobIdToStageIds.size === 3) - assert(listener.jobIdToSkippedStageIds.size === 3) - assert(listener.stageIdToJobId.size === 8) - assert(listener.stageIdToGraph.size === 8) - assert(listener.completedStageIds.size === 2) - - // Cleaning the third job should clean all of its stages - listener.cleanJob(startingJobId + 2) - assert(listener.jobIdToStageIds.size === 2) - assert(listener.jobIdToSkippedStageIds.size === 2) - assert(listener.stageIdToJobId.size === 6) - assert(listener.stageIdToGraph.size === 6) - assert(listener.completedStageIds.size === 2) - - // Cleaning one of the stages in the first job should clean that job and all of its stages - // Note that we still keep around the last stage because it belongs to a different job - listener.cleanStage(startingStageId) - assert(listener.jobIdToStageIds.size === 1) - assert(listener.jobIdToSkippedStageIds.size === 1) - assert(listener.stageIdToJobId.size === 1) - assert(listener.stageIdToGraph.size === 1) - assert(listener.completedStageIds.size === 0) - } - - /** Start a job with the specified number of stages. */ - private def startJob(numStages: Int, listener: RDDOperationGraphListener): Int = { - assert(numStages > 0, "I will not run a job with 0 stages for you.") - val stageInfos = (0 until numStages).map { _ => - val stageInfo = new StageInfo(stageIdCounter, 0, "s", 0, Seq.empty, Seq.empty, "d") - stageIdCounter += 1 - stageInfo - } - val jobId = jobIdCounter - listener.onJobStart(new SparkListenerJobStart(jobId, 0, stageInfos)) - // Also start all stages that belong to this job - stageInfos.map(_.stageId).foreach { sid => startStage(sid, listener) } - jobIdCounter += 1 - jobId - } - - /** Start the stage specified by the given ID. */ - private def startStage(stageId: Int, listener: RDDOperationGraphListener): Unit = { - val stageInfo = new StageInfo(stageId, 0, "s", 0, Seq.empty, Seq.empty, "d") - listener.onStageSubmitted(new SparkListenerStageSubmitted(stageInfo)) - } - - /** Finish the stage specified by the given ID. */ - private def endStage(stageId: Int, listener: RDDOperationGraphListener): Unit = { - val stageInfo = new StageInfo(stageId, 0, "s", 0, Seq.empty, Seq.empty, "d") - listener.onStageCompleted(new SparkListenerStageCompleted(stageInfo)) - } - - /** Finish the job specified by the given ID. */ - private def endJob(jobId: Int, listener: RDDOperationGraphListener): Unit = { - listener.onJobEnd(new SparkListenerJobEnd(jobId, 0, JobSucceeded)) - } - -} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 0c31b2b4a940..d76bf4dc44db 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -41,6 +41,8 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.getSparkUI"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.env.EnvironmentListener"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ui.exec.ExecutorsListener"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ExecutorStageSummary.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.JobData.this"), // [SPARK-20495][SQL] Add StorageLevel to cacheTable API ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.cacheTable"), From 9617db88e00fc965cc2f5ba84fecb109f2a7117b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 13 Nov 2017 11:16:18 -0800 Subject: [PATCH 2/6] Cleanup imports. --- .../main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 4 +--- core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala | 2 +- .../src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala | 2 -- .../src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../main/scala/org/apache/spark/ui/jobs/StageTable.scala | 6 +----- 5 files changed, 4 insertions(+), 12 deletions(-) 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 617ba05bc414..3ee3bfc800de 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 @@ -22,8 +22,7 @@ import java.util.Date import javax.servlet.http.HttpServletRequest import scala.collection.JavaConverters._ -import scala.collection.mutable.{HashMap, ListBuffer} -import scala.util.Try +import scala.collection.mutable.ListBuffer import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils @@ -33,7 +32,6 @@ import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1 import org.apache.spark.ui._ -import org.apache.spark.ui.jobs.UIData.{JobUIData, StageUIData} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished jobs */ 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 15c51f7efc4c..dd28df068891 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 @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -import java.util.{Date, Locale} +import java.util.Locale import javax.servlet.http.HttpServletRequest import scala.collection.mutable.{Buffer, ListBuffer} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index ddc42eea2670..5dfce858dec0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -19,12 +19,10 @@ package org.apache.spark.ui.jobs import java.net.URLEncoder -import scala.collection.mutable.HashMap import scala.xml.Node import org.apache.spark.scheduler.Schedulable import org.apache.spark.status.PoolData -import org.apache.spark.status.api.v1.StageData import org.apache.spark.ui.UIUtils /** Table showing list of pools */ 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 e3db8adad94f..51d1c174fa75 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 @@ -28,7 +28,7 @@ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.SparkConf import org.apache.spark.internal.config._ -import org.apache.spark.scheduler.{TaskInfo, TaskLocality} +import org.apache.spark.scheduler.TaskLocality import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1._ import org.apache.spark.ui._ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 5a488e61da29..18a4926f2f6c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -17,7 +17,6 @@ package org.apache.spark.ui.jobs -import java.lang.{Long => JLong} import java.net.URLEncoder import java.util.Date import javax.servlet.http.HttpServletRequest @@ -27,12 +26,9 @@ import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils -import org.apache.spark.scheduler.StageInfo -import org.apache.spark.status.{AppStatusStore, TaskDataWrapper} +import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1 -import org.apache.spark.storage.RDDInfo import org.apache.spark.ui._ -import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils private[ui] class StageTableBase( From 5f848c67e89e49b99fb4bcd95da2be1de1591a42 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 13 Nov 2017 11:23:22 -0800 Subject: [PATCH 3/6] Remove unnecessary reverse. --- .../scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) 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 3ee3bfc800de..b60d39b21b4b 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 @@ -275,23 +275,20 @@ private[ui] class AllJobsPage(parent: JobsTab, store: AppStatusStore) extends We val endTime = appInfo.attempts.head.endTime.getTime() val activeJobs = new ListBuffer[v1.JobData]() - val _completedJobs = new ListBuffer[v1.JobData]() - val _failedJobs = new ListBuffer[v1.JobData]() + val completedJobs = new ListBuffer[v1.JobData]() + val failedJobs = new ListBuffer[v1.JobData]() store.jobsList(null).foreach { job => job.status match { case JobExecutionStatus.SUCCEEDED => - _completedJobs += job + completedJobs += job case JobExecutionStatus.FAILED => - _failedJobs += job + failedJobs += job case _ => activeJobs += job } } - val completedJobs = _completedJobs.toSeq.reverse - val failedJobs = _failedJobs.toSeq.reverse - val activeJobsTable = jobsTable(request, "active", "activeJob", activeJobs, killEnabled = parent.killEnabled) val completedJobsTable = @@ -465,6 +462,7 @@ private[ui] class JobDataSource( } } + private[ui] class JobPagedTable( store: AppStatusStore, data: Seq[v1.JobData], From 1734dc0ddcac5470fbf4e15ec935c46fa81ca5b2 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 13 Nov 2017 11:47:42 -0800 Subject: [PATCH 4/6] Centralize handling of NoSuchElementException -> Option. --- .../apache/spark/status/AppStatusStore.scala | 21 ++++++++++++------- .../apache/spark/ui/jobs/AllStagesPage.scala | 8 ++----- .../apache/spark/ui/jobs/ExecutorTable.scala | 5 +---- .../org/apache/spark/ui/jobs/JobPage.scala | 5 +---- .../org/apache/spark/ui/jobs/JobsTab.scala | 5 +---- .../org/apache/spark/ui/jobs/PoolPage.scala | 10 ++------- .../org/apache/spark/ui/jobs/StagePage.scala | 19 +++++++---------- .../org/apache/spark/ui/jobs/StagesTab.scala | 7 ++----- 8 files changed, 30 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 809bd15668fd..c2764d5bef02 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -66,13 +66,8 @@ private[spark] class AppStatusStore(store: KVStore) { filtered.asScala.map(_.info).toSeq } - def executorSummary(executorId: String): Option[v1.ExecutorSummary] = { - try { - Some(store.read(classOf[ExecutorSummaryWrapper], executorId).info) - } catch { - case _: NoSuchElementException => - None - } + def executorSummary(executorId: String): v1.ExecutorSummary = { + store.read(classOf[ExecutorSummaryWrapper], executorId).info } def stageList(statuses: JList[v1.StageStatus]): Seq[v1.StageData] = { @@ -236,6 +231,18 @@ private[spark] class AppStatusStore(store: KVStore) { }.toSeq } + /** + * Calls a closure that may throw a NoSuchElementException and returns `None` when the exception + * is thrown. + */ + def asOption[T](fn: => T): Option[T] = { + try { + Some(fn) + } catch { + case _: NoSuchElementException => None + } + } + private def stageWithDetails(stage: v1.StageData): v1.StageData = { val tasks = taskList(stage.stageId, stage.attemptId, Int.MaxValue) .map { t => (t.taskId, t) } 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 78381754bd1b..e4cf99e7b9e0 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 @@ -58,12 +58,8 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { // For now, pool information is only accessible in live UIs val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]).map { pool => - val uiPool = try { - parent.store.pool(pool.name) - } catch { - case _: NoSuchElementException => - new PoolData(pool.name, Set()) - } + val uiPool = parent.store.asOption(parent.store.pool(pool.name)).getOrElse( + new PoolData(pool.name, Set())) pool -> uiPool }.toMap val poolTable = new PoolTable(pools, parent) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 12b3b99ed1a9..41d42b52430a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -17,9 +17,6 @@ package org.apache.spark.ui.jobs -import java.util.NoSuchElementException - -import scala.collection.mutable import scala.xml.{Node, Unparsed} import org.apache.spark.status.AppStatusStore @@ -91,7 +88,7 @@ private[ui] class ExecutorTable(stage: StageData, store: AppStatusStore) { private def createExecutorTable(stage: StageData) : Seq[Node] = { stage.executorSummary.getOrElse(Map.empty).toSeq.sortBy(_._1).map { case (k, v) => - val executor = store.executorSummary(k) + val executor = store.asOption(store.executorSummary(k))
{k}
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 dd28df068891..c4a45ed2732f 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 @@ -21,7 +21,6 @@ import java.util.Locale import javax.servlet.http.HttpServletRequest import scala.collection.mutable.{Buffer, ListBuffer} -import scala.util.Try import scala.xml.{Node, NodeSeq, Unparsed, Utility} import org.apache.commons.lang3.StringEscapeUtils @@ -190,8 +189,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") val jobId = parameterId.toInt - val jobDataOption = Try(store.job(jobId)).toOption - if (jobDataOption.isEmpty) { + val jobData = store.asOption(store.job(jobId)).getOrElse { val content =

No information to display for job {jobId}

@@ -199,7 +197,6 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP return UIUtils.headerSparkPage( s"Details for Job $jobId", content, parent) } - val jobData = jobDataOption.get val isComplete = jobData.status != JobExecutionStatus.RUNNING val stages = jobData.stageIds.map { stageId => // This could be empty if the listener hasn't received information about the diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index d1c498c32b5f..0c5f27b5724f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -54,8 +54,7 @@ private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore) // stripXSS is called first to remove suspicious characters used in XSS attacks val jobId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) jobId.foreach { id => - try { - val job = store.job(id) + store.asOption(store.job(id)).foreach { job => if (job.status == JobExecutionStatus.RUNNING) { sc.foreach(_.cancelJob(id)) // Do a quick pause here to give Spark time to kill the job so it shows up as @@ -63,8 +62,6 @@ private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore) // time should be limited in duration. Thread.sleep(100) } - } catch { - case _: NoSuchElementException => } } } 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 1f125b70e0fb..98fbd7aceaa1 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 @@ -17,7 +17,6 @@ package org.apache.spark.ui.jobs -import java.util.NoSuchElementException import javax.servlet.http.HttpServletRequest import scala.xml.Node @@ -42,13 +41,8 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { throw new IllegalArgumentException(s"Unknown pool: $poolName") } - val uiPool = try { - parent.store.pool(poolName) - } catch { - case _: NoSuchElementException => - new PoolData(poolName, Set()) - } - + val uiPool = parent.store.asOption(parent.store.pool(poolName)).getOrElse( + new PoolData(poolName, Set())) val activeStages = uiPool.stageIds.toSeq.map(parent.store.lastStageAttempt(_)) val activeStagesTable = new StageTableBase(parent.store, request, activeStages, "", "activeStage", parent.basePath, 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 51d1c174fa75..521c9fbbcdd2 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 @@ -18,7 +18,7 @@ package org.apache.spark.ui.jobs import java.net.URLEncoder -import java.util.{Date, NoSuchElementException} +import java.util.Date import javax.servlet.http.HttpServletRequest import scala.collection.mutable.{HashMap, HashSet} @@ -108,16 +108,15 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val stageAttemptId = parameterAttempt.toInt val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" - val stageData = try { - parent.store.stageAttempt(stageId, stageAttemptId, details = true) - } catch { - case _: NoSuchElementException => + val stageData = parent.store + .asOption(parent.store.stageAttempt(stageId, stageAttemptId, details = true)) + .getOrElse { val content =

No information to display for Stage {stageId} (Attempt {stageAttemptId})

return UIUtils.headerSparkPage(stageHeader, content, parent) - } + } val tasks = stageData.tasks.getOrElse(Map.empty).values.toSeq if (tasks.isEmpty) { @@ -258,11 +257,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We
- val stageGraph = try { - Some(parent.store.operationGraphForStage(stageId)) - } catch { - case _: NoSuchElementException => None - } + val stageGraph = parent.store.asOption(parent.store.operationGraphForStage(stageId)) val dagViz = UIUtils.showDagVizForStage(stageId, stageGraph) val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") @@ -1012,7 +1007,7 @@ private[ui] class TaskDataSource( private def executorLogs(id: String): Map[String, String] = { executors.getOrElseUpdate(id, - store.executorSummary(id).map(_.executorLogs).getOrElse(Map.empty)) + store.asOption(store.executorSummary(id)).map(_.executorLogs).getOrElse(Map.empty)) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 7631ee2d0cfe..be05a963f0e6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -17,7 +17,6 @@ package org.apache.spark.ui.jobs -import java.util.NoSuchElementException import javax.servlet.http.HttpServletRequest import org.apache.spark.scheduler.SchedulingMode @@ -49,8 +48,8 @@ private[ui] class StagesTab(val parent: SparkUI, val store: AppStatusStore) // stripXSS is called first to remove suspicious characters used in XSS attacks val stageId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) stageId.foreach { id => - try { - val status = store.lastStageAttempt(id).status + store.asOption(store.lastStageAttempt(id)).foreach { stage => + val status = stage.status if (status == StageStatus.ACTIVE || status == StageStatus.PENDING) { sc.foreach(_.cancelStage(id, "killed via the Web UI")) // Do a quick pause here to give Spark time to kill the stage so it shows up as @@ -58,8 +57,6 @@ private[ui] class StagesTab(val parent: SparkUI, val store: AppStatusStore) // time should be limited in duration. Thread.sleep(100) } - } catch { - case _: NoSuchElementException => } } } From 1fef09da600141593942154b312a8bf100d6ea5c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 13 Nov 2017 11:56:10 -0800 Subject: [PATCH 5/6] Other feedback. --- .../scala/org/apache/spark/ui/jobs/JobPage.scala | 2 +- .../scala/org/apache/spark/ui/jobs/JobsTab.scala | 12 ++++-------- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 4 ++-- 3 files changed, 7 insertions(+), 11 deletions(-) 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 c4a45ed2732f..740f12e7d13d 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 @@ -334,7 +334,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP val appStartTime = store.applicationInfo().attempts.head.startTime.getTime() content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, - store.executorList(false), appStartTime) + store.executorList(false), appStartTime) content ++= UIUtils.showDagVizForJob( jobId, store.operationGraphForJob(jobId)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index 0c5f27b5724f..99eab1b2a27d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -34,14 +34,10 @@ private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore) val killEnabled = parent.killEnabled def isFairScheduler: Boolean = { - val configName = "spark.scheduler.mode" - val config = sc match { - case Some(_sc) => - _sc.conf.getOption(configName) - case _ => - store.environmentInfo().sparkProperties.toMap.get(configName) - } - config.map(SchedulingMode.withName) == Some(SchedulingMode.FAIR) + store.environmentInfo().sparkProperties.toMap + .get("spark.scheduler.mode") + .map { mode => mode == SchedulingMode.FAIR } + .getOrElse(false) } def getSparkUser: String = parent.getSparkUser 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 521c9fbbcdd2..5f93f2ffb412 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 @@ -842,7 +842,7 @@ private[ui] class TaskDataSource( import StagePage._ // Keep an internal cache of executor log maps so that long task lists render faster. - private val executors = new HashMap[String, Map[String, String]]() + private val executorIdToLogs = new HashMap[String, Map[String, String]]() // Convert TaskData to TaskTableRowData which contains the final contents to show in the table // so that we can avoid creating duplicate contents during sorting the data @@ -1006,7 +1006,7 @@ private[ui] class TaskDataSource( } private def executorLogs(id: String): Map[String, String] = { - executors.getOrElseUpdate(id, + executorIdToLogs.getOrElseUpdate(id, store.asOption(store.executorSummary(id)).map(_.executorLogs).getOrElse(Map.empty)) } From 0454ed1ad5bf36cf6c6f5ebdf899298daf2a50eb Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 13 Nov 2017 12:34:59 -0800 Subject: [PATCH 6/6] Indentation. --- .../src/main/scala/org/apache/spark/status/AppStatusStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index c2764d5bef02..9b42f5560575 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -67,7 +67,7 @@ private[spark] class AppStatusStore(store: KVStore) { } def executorSummary(executorId: String): v1.ExecutorSummary = { - store.read(classOf[ExecutorSummaryWrapper], executorId).info + store.read(classOf[ExecutorSummaryWrapper], executorId).info } def stageList(statuses: JList[v1.StageStatus]): Seq[v1.StageData] = {