diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index e5aaaf6c155e..1d325e651b1d 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)
@@ -449,11 +449,7 @@ class SparkContext(config: SparkConf) extends Logging {
_ui =
if (conf.getBoolean("spark.ui.enabled", true)) {
- Some(SparkUI.create(Some(this), _statusStore, _conf,
- l => listenerBus.addToStatusQueue(l),
- _env.securityManager,
- appName,
- "",
+ Some(SparkUI.create(Some(this), _statusStore, _conf, _env.securityManager, appName, "",
startTime))
} else {
// For tests, do not enable the UI
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..a6dc53321d65 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 {
@@ -324,13 +325,10 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
}
val loadedUI = {
- val ui = SparkUI.create(None, new AppStatusStore(kvstore), conf,
- l => replayBus.addListener(l),
- secManager,
- app.info.name,
+ val ui = SparkUI.create(None, new AppStatusStore(kvstore), conf, secManager, app.info.name,
HistoryServer.getAttemptURI(appId, attempt.info.attemptId),
attempt.info.startTime.getTime(),
- appSparkVersion = attempt.info.appSparkVersion)
+ attempt.info.appSparkVersion)
LoadedAppUI(ui)
}
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 7f2c00c09d43..f2d8e0a5480b 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
@@ -210,16 +218,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)
@@ -234,17 +241,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)
}
}
@@ -262,12 +303,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)
@@ -279,7 +332,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)
@@ -318,6 +371,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
@@ -337,11 +392,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 =>
@@ -350,13 +409,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)
}
@@ -364,6 +439,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)
}
@@ -422,6 +498,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)
}
@@ -482,11 +563,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 = {
@@ -628,6 +713,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 80c8d7d11a3c..9b42f5560575 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
@@ -65,31 +66,40 @@ 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] = {
- 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 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): v1.StageData = {
- store.read(classOf[StageDataWrapper], Array(stageId, stageAttemptId)).info
+ 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 +199,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,
@@ -215,6 +231,66 @@ 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) }
+ .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
}
@@ -223,6 +299,27 @@ private[spark] class AppStatusStore(store: KVStore) {
store.view(classOf[StreamBlockData]).asScala.toSeq
}
+ 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()
}
@@ -237,12 +334,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 706d94c3a59b..ef2936c9b69a 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)
}
@@ -535,6 +572,16 @@ private class LiveRDD(val 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 f44b7935bfaa..c1ea87542d6c 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) {
@@ -147,3 +153,37 @@ private[spark] class StreamBlockData(
def key: Array[String] = Array(name, executorId)
}
+
+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 e93ade001c60..35da3c3bfd1a 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
@@ -29,8 +29,7 @@ import org.apache.spark.status.api.v1._
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.StorageTab
import org.apache.spark.util.Utils
@@ -42,11 +41,8 @@ private[spark] class SparkUI private (
val sc: Option[SparkContext],
val conf: SparkConf,
securityManager: SecurityManager,
- val jobProgressListener: JobProgressListener,
- 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),
@@ -61,8 +57,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)
@@ -72,6 +68,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")))
@@ -79,6 +76,7 @@ private[spark] class SparkUI private (
"/stages/stage/kill", "/stages/", stagesTab.handleKillRequest,
httpMethods = Set("GET", "POST")))
}
+
initialize()
def getSparkUser: String = {
@@ -170,25 +168,13 @@ private[spark] object SparkUI {
sc: Option[SparkContext],
store: AppStatusStore,
conf: SparkConf,
- addListenerFn: SparkListenerInterface => Unit,
securityManager: SecurityManager,
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 operationGraphListener = new RDDOperationGraphListener(conf)
-
- addListenerFn(operationGraphListener)
-
- new SparkUI(store, sc, conf, securityManager, jobProgressListener, operationGraphListener,
- appName, basePath, lastUpdateTime, startTime, appSparkVersion)
+ new SparkUI(store, sc, conf, securityManager, 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..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
@@ -22,20 +22,20 @@ import java.util.Date
import javax.servlet.http.HttpServletRequest
import scala.collection.JavaConverters._
-import scala.collection.mutable.{HashMap, ListBuffer}
+import scala.collection.mutable.ListBuffer
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 +110,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 +155,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 +203,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 +244,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 +270,117 @@ 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 =
-
-
-
- User:
- {parent.getSparkUser}
-
-
- Total Uptime:
- {
- if (endTime < 0 && parent.sc.isDefined) {
- UIUtils.formatDuration(System.currentTimeMillis() - startTime)
- } else if (endTime > 0) {
- UIUtils.formatDuration(endTime - startTime)
- }
- }
-
-
- Scheduling Mode:
- {listener.schedulingMode.map(_.toString).getOrElse("Unknown")}
-
+ 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 = s"${completedJobs.size}"
+ val schedulingMode = store.environmentInfo().sparkProperties.toMap
+ .get("spark.scheduler.mode")
+ .map { mode => SchedulingMode.withName(mode).toString }
+ .getOrElse("Unknown")
+
+ val summary: NodeSeq =
+
+
+
+ User:
+ {parent.getSparkUser}
+
+
+ Total Uptime:
{
- if (shouldShowActiveJobs) {
-
- Active Jobs:
- {activeJobs.size}
-
+ if (endTime < 0 && parent.sc.isDefined) {
+ UIUtils.formatDuration(System.currentTimeMillis() - startTime)
+ } else if (endTime > 0) {
+ UIUtils.formatDuration(endTime - startTime)
}
}
- {
- if (shouldShowCompletedJobs) {
-
- Completed Jobs:
- {completedJobNumStr}
-
- }
+
+
+ Scheduling Mode:
+ {schedulingMode}
+
+ {
+ if (shouldShowActiveJobs) {
+
+ Active Jobs:
+ {activeJobs.size}
+
}
- {
- if (shouldShowFailedJobs) {
-
- Failed Jobs:
- {listener.numFailedJobs}
-
- }
+ }
+ {
+ if (shouldShowCompletedJobs) {
+
+ Completed Jobs:
+ {completedJobNumStr}
+
}
-
-
+ }
+ {
+ if (shouldShowFailedJobs) {
+
+ Failed Jobs:
+ {failedJobs.size}
+
+ }
+ }
+
+
- 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 +391,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 +413,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
@@ -479,15 +462,15 @@ 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 +493,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 +606,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..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
@@ -22,120 +22,121 @@ 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 = parent.store.asOption(parent.store.pool(pool.name)).getOrElse(
+ new PoolData(pool.name, Set()))
+ pool -> uiPool
+ }.toMap
+ val poolTable = new PoolTable(pools, parent)
+
+ val shouldShowActiveStages = activeStages.nonEmpty
+ val shouldShowPendingStages = pendingStages.nonEmpty
+ val shouldShowCompletedStages = completedStages.nonEmpty
+ val shouldShowFailedStages = failedStages.nonEmpty
+
+ val completedStageNumStr = if (numCompletedStages == completedStages.size) {
+ s"$numCompletedStages"
+ } else {
+ s"$numCompletedStages, only showing ${completedStages.size}"
+ }
- val summary: NodeSeq =
-
-
- {
- if (shouldShowActiveStages) {
-
- Active Stages:
- {activeStages.size}
-
- }
+ val summary: NodeSeq =
+
+
+ {
+ if (shouldShowActiveStages) {
+
+ Active Stages:
+ {activeStages.size}
+
}
- {
- if (shouldShowPendingStages) {
-
- Pending Stages:
- {pendingStages.size}
-
- }
+ }
+ {
+ if (shouldShowPendingStages) {
+
+ Pending Stages:
+ {pendingStages.size}
+
}
- {
- if (shouldShowCompletedStages) {
-
- Completed Stages:
- {completedStageNumStr}
-
- }
+ }
+ {
+ if (shouldShowCompletedStages) {
+
+ Completed Stages:
+ {completedStageNumStr}
+
}
- {
- if (shouldShowFailedStages) {
-
- Failed Stages:
- {numFailedStages}
-
- }
+ }
+ {
+ if (shouldShowFailedStages) {
+
+ Failed Stages:
+ {numFailedStages}
+
}
-
-
-
- var content = summary ++
- {
- if (sc.isDefined && isFairScheduler) {
- Fair Scheduler Pools ({pools.size}) ++ poolTable.toNodeSeq
- } else {
- Seq.empty[Node]
}
+
+
+
+ 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..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,44 +17,19 @@
package org.apache.spark.ui.jobs
-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
+private[ui] class ExecutorTable(stage: StageData, store: AppStatusStore) {
- def toNodeSeq: Seq[Node] = {
- listener.synchronized {
- executorTable()
- }
- }
-
- /** 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] = {
Executor ID
@@ -64,29 +39,29 @@ private[ui] class ExecutorTable(
Failed Tasks
Killed Tasks
Succeeded Tasks
- {if (hasInput) {
+ {if (hasInput(stage)) {
Input Size / Records
}}
- {if (hasOutput) {
+ {if (hasOutput(stage)) {
Output Size / Records
}}
- {if (hasShuffleRead) {
+ {if (hasShuffleRead(stage)) {
Shuffle Read Size / Records
}}
- {if (hasShuffleWrite) {
+ {if (hasShuffleWrite(stage)) {
Shuffle Write Size / Records
}}
- {if (hasBytesSpilled) {
+ {if (hasBytesSpilled(stage)) {
Shuffle Spill (Memory)
Shuffle Spill (Disk)
}}
@@ -97,7 +72,7 @@ private[ui] class ExecutorTable(
- {createExecutorTable()}
+ {createExecutorTable(stage)}
}
- 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) =>
- }
- }
-
-
- {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.asOption(store.executorSummary(k))
+
+
+ {k}
+
+ {
+ executor.map(_.executorLogs).getOrElse(Map.empty).map {
+ case (logName, logUrl) =>
+ }
+ }
+
+
+ {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..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
@@ -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}
@@ -27,11 +27,12 @@ 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 +57,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 +81,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 +95,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 +139,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 +184,181 @@ 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 jobData = store.asOption(store.job(jobId)).getOrElse {
+ val content =
+
+
No information to display for job {jobId}
+
+ return UIUtils.headerSparkPage(
+ s"Details for Job $jobId", content, parent)
+ }
+ 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 =
-
-
-
- Status:
- {jobData.status}
-
- {
- if (jobData.jobGroup.isDefined) {
-
- Job Group:
- {jobData.jobGroup.get}
-
- }
- }
- {
- if (shouldShowActiveStages) {
-
- Active Stages:
- {activeStages.size}
-
- }
- }
- {
- if (shouldShowPendingStages) {
-
-
- Pending Stages:
- {pendingOrSkippedStages.size}
-
- }
+ val summary: NodeSeq =
+
+
+
+ Status:
+ {jobData.status}
+
+ {
+ if (jobData.jobGroup.isDefined) {
+
+ Job Group:
+ {jobData.jobGroup.get}
+
}
- {
- if (shouldShowCompletedStages) {
-
- Completed Stages:
- {completedStages.size}
-
- }
+ }
+ {
+ if (shouldShowActiveStages) {
+
+ Active Stages:
+ {activeStages.size}
+
}
- {
- if (shouldShowSkippedStages) {
+ }
+ {
+ if (shouldShowPendingStages) {
- Skipped Stages:
- {pendingOrSkippedStages.size}
+
+ Pending Stages:
+ {pendingOrSkippedStages.size}
}
+ }
+ {
+ if (shouldShowCompletedStages) {
+
+ Completed Stages:
+ {completedStages.size}
+
}
- {
- if (shouldShowFailedStages) {
-
- Failed Stages:
- {failedStages.size}
-
- }
+ }
+ {
+ if (shouldShowSkippedStages) {
+
+ Skipped Stages:
+ {pendingOrSkippedStages.size}
+
+ }
+ }
+ {
+ if (shouldShowFailedStages) {
+
+ Failed Stages:
+ {failedStages.size}
+
}
-
-
+ }
+
+
- 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..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
@@ -19,35 +19,45 @@ 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 = {
+ store.environmentInfo().sparkProperties.toMap
+ .get("spark.scheduler.mode")
+ .map { mode => mode == SchedulingMode.FAIR }
+ .getOrElse(false)
+ }
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)
+ 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
+ // killed after the refresh. Note that this will block the serving thread so the
+ // time should be limited in duration.
+ Thread.sleep(100)
+ }
}
}
}
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..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
@@ -21,46 +21,39 @@ 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")
- }
-
- 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 pools = sc.map(_.getPoolForName(poolName).getOrElse {
- throw new IllegalArgumentException(s"Unknown poolname: $poolName")
- }).toSeq
- val poolTable = new PoolTable(pools, parent)
+ // 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")
+ }
- var content = Summary ++ poolTable.toNodeSeq
- if (shouldShowActiveStages) {
- content ++= Active Stages ({activeStages.size}) ++ activeStagesTable.toNodeSeq
- }
+ // 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")
+ }
- UIUtils.headerSparkPage("Fair Scheduler Pool: " + poolName, content, parent)
+ 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,
+ "stages/pool", parent.isFairScheduler, parent.killEnabled, false)
+
+ 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..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,25 +19,16 @@ 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, StageInfo}
+import org.apache.spark.scheduler.Schedulable
+import org.apache.spark.status.PoolData
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] = {
Pool Name
@@ -48,29 +39,24 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) {
SchedulingMode
- {rows.map(r => makeRow(r, listener.poolToActiveStages))}
+ {pools.map { case (s, p) => poolRow(s, p) }}
}
- 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..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
@@ -21,26 +21,25 @@ import java.net.URLEncoder
import java.util.Date
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.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,521 @@ 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 = 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)
-
- }
- 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
-
-
-
-
-
- (De)select All
-
-
-
-
- Scheduler Delay
-
-
-
-
-
- Task Deserialization Time
-
-
- {if (stageData.hasShuffleRead) {
-
-
-
- Shuffle Read Blocked Time
-
-
-
-
-
- Shuffle Remote Reads
-
-
- }}
-
-
-
- Result Serialization Time
-
-
+ val showAdditionalMetrics =
+
-
- val dagViz = UIUtils.showDagVizForStage(
- stageId, operationGraphListener.getOperationGraphForStage(stageId))
-
- val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value")
- def accumulableRow(acc: AccumulableInfo): Seq[Node] = {
- (acc.name, acc.value) match {
- case (Some(name), Some(value)) => {name} {value}
- case _ => Seq.empty[Node]
- }
- }
- 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,
- stageData.hasInput,
- stageData.hasOutput,
- stageData.hasShuffleRead,
- stageData.hasShuffleWrite,
- stageData.hasBytesSpilled,
- parent.lastUpdateTime,
- currentTime,
- pageSize = taskPageSize,
- sortColumn = taskSortColumn,
- desc = taskSortDesc,
- store = store
- )
- (_taskTable, _taskTable.table(page))
- } catch {
- case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) =>
- val errorMessage =
-
-
Error while rendering stage table:
-
- {Utils.exceptionString(e)}
-
-
- (null, errorMessage)
- }
-
- val jsForScrollingDownToTaskTable =
-
-
- val taskIdsInPage = if (taskTable == null) Set.empty[Long]
- else taskTable.dataSource.slicedTaskIds
-
- // Excludes tasks which failed and have incomplete metrics
- val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.metrics.isDefined)
-
- val summaryTable: Option[Seq[Node]] =
- if (validTasks.isEmpty) {
- 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)}
- }
- }
- def getFormattedSizeQuantiles(data: Seq[Double]): Seq[Elem] = {
- getDistributionQuantiles(data).map(d => {Utils.bytesToString(d.toLong)} )
- }
-
- val deserializationTimes = validTasks.map { taskUIData: TaskUIData =>
- taskUIData.metrics.get.executorDeserializeTime.toDouble
- }
- val deserializationQuantiles =
-
-
- Task Deserialization Time
-
- +: getFormattedTimeQuantiles(deserializationTimes)
-
- val serviceTimes = validTasks.map { taskUIData: TaskUIData =>
- taskUIData.metrics.get.executorRunTime.toDouble
- }
- val serviceQuantiles = Duration +: getFormattedTimeQuantiles(serviceTimes)
-
- val gcTimes = validTasks.map { taskUIData: TaskUIData =>
- taskUIData.metrics.get.jvmGCTime.toDouble
- }
- val gcQuantiles =
-
- GC Time
-
- +: getFormattedTimeQuantiles(gcTimes)
-
- val serializationTimes = validTasks.map { taskUIData: TaskUIData =>
- taskUIData.metrics.get.resultSerializationTime.toDouble
- }
- val serializationQuantiles =
-
+ }}
+
- Result Serialization Time
+
+ Result Serialization Time
- +: getFormattedTimeQuantiles(serializationTimes)
-
- val gettingResultTimes = validTasks.map { taskUIData: TaskUIData =>
- getGettingResultTime(taskUIData.taskInfo, currentTime).toDouble
- }
- val gettingResultQuantiles =
-
+
+
- Getting Result Time
+ title={ToolTips.GETTING_RESULT_TIME} data-placement="right">
+
+ Getting Result Time
- +:
- getFormattedTimeQuantiles(gettingResultTimes)
-
- val peakExecutionMemory = validTasks.map { taskUIData: TaskUIData =>
- taskUIData.metrics.get.peakExecutionMemory.toDouble
- }
- val peakExecutionMemoryQuantiles = {
-
+
+
- Peak Execution Memory
+
+ Peak Execution Memory
- +: getFormattedSizeQuantiles(peakExecutionMemory)
- }
-
- // 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 { taskUIData: TaskUIData =>
- getSchedulerDelay(taskUIData.taskInfo, taskUIData.metrics.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 inputSizes = validTasks.map { taskUIData: TaskUIData =>
- taskUIData.metrics.get.inputMetrics.bytesRead.toDouble
- }
-
- val inputRecords = validTasks.map { taskUIData: TaskUIData =>
- taskUIData.metrics.get.inputMetrics.recordsRead.toDouble
- }
-
- val inputQuantiles = Input Size / Records +:
- getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords)
+
+
+
+
- val outputSizes = validTasks.map { taskUIData: TaskUIData =>
- taskUIData.metrics.get.outputMetrics.bytesWritten.toDouble
- }
+ val stageGraph = parent.store.asOption(parent.store.operationGraphForStage(stageId))
+ 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 =
-
-
-
-
- {executorTable.toNodeSeq}
-
+ val executorTable = new ExecutorTable(stageData, parent.store)
+
+ val maybeAccumulableTable: Seq[Node] =
+ if (hasAccumulators) { Accumulators ++ accumulableTable } else Seq()
+
+ val aggMetrics =
+
+
+
+
+ {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) ++
- ++
- {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) ++
+ ++
+ {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 +594,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 +628,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 +670,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 +735,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 +797,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 +827,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 +841,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 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
private val data = tasks.map(taskRow).sorted(ordering(sortColumn, desc))
@@ -887,26 +860,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 +894,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 +977,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 +1001,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] = {
+ executorIdToLogs.getOrElseUpdate(id,
+ store.asOption(store.executorSummary(id)).map(_.executorLogs).getOrElse(Map.empty))
}
/**
@@ -1148,14 +1118,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 +1150,6 @@ private[ui] class TaskPagedTable(
hasShuffleRead,
hasShuffleWrite,
hasBytesSpilled,
- lastUpdateTime,
currentTime,
pageSize,
sortColumn,
@@ -1363,3 +1331,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..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
@@ -26,19 +26,19 @@ import scala.xml._
import org.apache.commons.lang3.StringEscapeUtils
-import org.apache.spark.scheduler.StageInfo
+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.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 +79,12 @@ private[ui] class StageTableBase(
val toNodeSeq = try {
new StagePagedTable(
+ store,
stages,
tableHeaderID,
stageTag,
basePath,
subPath,
- progressListener,
isFairScheduler,
killEnabled,
currentTime,
@@ -106,13 +106,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 +126,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 +165,8 @@ private[ui] class StagePagedTable(
parameterOtherTable.mkString("&")
override val dataSource = new StageDataSource(
+ store,
stages,
- listener,
currentTime,
pageSize,
sortColumn,
@@ -274,10 +275,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 +302,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 +319,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 +345,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 +369,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 +405,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 +425,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 +486,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..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
@@ -21,36 +21,42 @@ 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)
+ 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
+ // killed after the refresh. Note that this will block the serving thread so the
+ // time should be limited in duration.
+ Thread.sleep(100)
+ }
}
}
}
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.(