Skip to content

Commit

Permalink
Add progress bars to jobs page.
Browse files Browse the repository at this point in the history
  • Loading branch information
JoshRosen committed Nov 11, 2014
1 parent 45343b8 commit a475ea1
Show file tree
Hide file tree
Showing 5 changed files with 67 additions and 18 deletions.
13 changes: 13 additions & 0 deletions core/src/main/scala/org/apache/spark/ui/UIUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -283,4 +283,17 @@ private[spark] object UIUtils extends Logging {
</tbody>
</table>
}

def makeProgressBar(started: Int, completed: Int, failed: Int, total: Int): Seq[Node] = {
val completeWidth = "width: %s%%".format((completed.toDouble/total)*100)
val startWidth = "width: %s%%".format((started.toDouble/total)*100)

<div class="progress">
<span style="text-align:center; position:absolute; width:100%; left:0;">
{completed}/{total} { if (failed > 0) s"($failed failed)" else "" }
</span>
<div class="bar bar-completed" style={completeWidth}></div>
<div class="bar bar-running" style={startWidth}></div>
</div>
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
<th>Description</th>
<th>Submitted</th>
<th>Duration</th>
<th>Tasks: Succeeded/Total</th>
}

def makeRow(job: JobUIData): Seq[Node] = {
Expand Down Expand Up @@ -74,6 +75,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
{formattedSubmissionTime}
</td>
<td sorttable_customkey={duration.getOrElse(-1).toString}>{formattedDuration}</td>
<td class="progress-cell">
{UIUtils.makeProgressBar(job.numActiveTasks, job.numCompletedTasks,
job.numFailedTasks, job.numTasks)}
</td>
</tr>
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.ui.jobs

import scala.collection.mutable.{HashMap, ListBuffer}
import scala.collection.mutable.{HashMap, HashSet, ListBuffer}

import org.apache.spark._
import org.apache.spark.annotation.DeveloperApi
Expand Down Expand Up @@ -59,7 +59,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
val failedStages = ListBuffer[StageInfo]()
val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData]
val stageIdToInfo = new HashMap[StageId, StageInfo]

val stageIdToActiveJobIds = new HashMap[StageId, HashSet[JobId]]

// Number of completed and failed stages, may not actually equal to completedStages.size and
// failedStages.size respectively due to completedStage and failedStages only maintain the latest
// part of the stages, the earlier ones will be removed when there are too many stages for
Expand All @@ -86,6 +87,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
jobGroup, JobExecutionStatus.RUNNING)
jobIdToData(jobStart.jobId) = jobData
activeJobs(jobStart.jobId) = jobData
for (stageId <- jobStart.stageIds) {
stageIdToActiveJobIds.getOrElseUpdate(stageId, new HashSet[StageId]).add(jobStart.jobId)
}
}

override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized {
Expand All @@ -102,6 +106,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
failedJobs += jobData
jobData.status = JobExecutionStatus.FAILED
}
for (stageId <- jobData.stageIds) {
stageIdToActiveJobIds.get(stageId).foreach(_.remove(jobEnd.jobId))
}
}

override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized {
Expand Down Expand Up @@ -138,6 +145,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
stages.take(toRemove).foreach { s =>
stageIdToData.remove((s.stageId, s.attemptId))
stageIdToInfo.remove(s.stageId)
stageIdToActiveJobIds.remove(s.stageId)
}
stages.trimStart(toRemove)
}
Expand All @@ -162,6 +170,14 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {

val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo])
stages(stage.stageId) = stage

for (
activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId);
jobId <- activeJobsDependentOnStage;
jobData <- jobIdToData.get(jobId)
) {
jobData.numTasks += stage.numTasks
}
}

override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized {
Expand All @@ -174,6 +190,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
stageData.numActiveTasks += 1
stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo))
}
for (
activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId);
jobId <- activeJobsDependentOnStage;
jobData <- jobIdToData.get(jobId)
) {
jobData.numActiveTasks += 1
}
}

override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) {
Expand Down Expand Up @@ -208,6 +231,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
execSummary.taskTime += info.duration
stageData.numActiveTasks -= 1

val isRecomputation = stageData.completedIndices.contains(info.index)

val (errorMessage, metrics): (Option[String], Option[TaskMetrics]) =
taskEnd.reason match {
case org.apache.spark.Success =>
Expand All @@ -231,6 +256,22 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
taskData.taskInfo = info
taskData.taskMetrics = metrics
taskData.errorMessage = errorMessage

for (
activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskEnd.stageId);
jobId <- activeJobsDependentOnStage;
jobData <- jobIdToData.get(jobId)
) {
jobData.numActiveTasks -= 1
taskEnd.reason match {
case Success =>
if (!isRecomputation) {
jobData.numCompletedTasks += 1
}
case _ =>
jobData.numFailedTasks += 1
}
}
}
}

Expand Down
16 changes: 1 addition & 15 deletions core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
Original file line number Diff line number Diff line change
Expand Up @@ -69,20 +69,6 @@ private[ui] class StageTableBase(
</table>
}

private def makeProgressBar(started: Int, completed: Int, failed: Int, total: Int): Seq[Node] =
{
val completeWidth = "width: %s%%".format((completed.toDouble/total)*100)
val startWidth = "width: %s%%".format((started.toDouble/total)*100)

<div class="progress">
<span style="text-align:center; position:absolute; width:100%; left:0;">
{completed}/{total} { if (failed > 0) s"($failed failed)" else "" }
</span>
<div class="bar bar-completed" style={completeWidth}></div>
<div class="bar bar-running" style={startWidth}></div>
</div>
}

private def makeDescription(s: StageInfo): Seq[Node] = {
// scalastyle:off
val killLink = if (killEnabled) {
Expand Down Expand Up @@ -172,7 +158,7 @@ private[ui] class StageTableBase(
<td valign="middle">{submissionTime}</td>
<td sorttable_customkey={duration.getOrElse(-1).toString}>{formattedDuration}</td>
<td class="progress-cell">
{makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size,
{UIUtils.makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size,
stageData.numFailedTasks, s.numTasks)}
</td>
<td sorttable_customkey={inputRead.toString}>{inputReadWithUnit}</td>
Expand Down
6 changes: 5 additions & 1 deletion core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,11 @@ private[jobs] object UIData {
var endTime: Option[Long] = None,
var stageIds: Seq[Int] = Seq.empty,
var jobGroup: Option[String] = None,
var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN
var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN,
var numTasks: Int = 0,
var numActiveTasks: Int = 0,
var numCompletedTasks: Int = 0,
var numFailedTasks: Int = 0
)

class StageUIData {
Expand Down

0 comments on commit a475ea1

Please sign in to comment.