From 2568a6cfcf5287e29d0e4be58280aa7b934b01a7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 29 Oct 2014 15:27:16 -0700 Subject: [PATCH 01/35] Rename JobProgressPage to AllStagesPage: This will avoid confusion once we have a page that lists all jobs. --- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 4 ++-- .../ui/jobs/{JobProgressPage.scala => AllStagesPage.scala} | 2 +- .../main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala | 2 +- core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala | 2 +- .../src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala | 2 +- .../src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../main/scala/org/apache/spark/ui/jobs/StageTable.scala | 4 ++-- .../spark/ui/jobs/{JobProgressTab.scala => StagesTab.scala} | 6 +++--- 8 files changed, 12 insertions(+), 12 deletions(-) rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressPage.scala => AllStagesPage.scala} (97%) rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressTab.scala => StagesTab.scala} (91%) 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 049938f82729..9427ee9069c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -23,7 +23,7 @@ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} -import org.apache.spark.ui.jobs.{JobProgressListener, JobProgressTab} +import org.apache.spark.ui.jobs.{JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} /** @@ -45,7 +45,7 @@ private[spark] class SparkUI private ( /** Initialize all components of the server. */ def initialize() { - val jobProgressTab = new JobProgressTab(this) + val jobProgressTab = new StagesTab(this) attachTab(jobProgressTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 6e718eecdd52..3b7d55361e44 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { +private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { private val sc = parent.sc private val listener = parent.listener private def isFairScheduler = parent.isFairScheduler 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 f0e43fbf7097..c8316276fedd 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 @@ -25,7 +25,7 @@ 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: JobProgressTab) { +private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StagesTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { 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 770d99eea1c9..968aaba301c7 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 @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { +private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { private val sc = parent.sc private val listener = parent.listener 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 64178e1e33d4..df1899e7a9b8 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 @@ -24,7 +24,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { 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 2414e4c65237..a84d59b75805 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.scheduler.AccumulableInfo /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { +private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { 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 4ee7f08ab47a..63a163d3c2b0 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 @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTableBase( stages: Seq[StageInfo], - parent: JobProgressTab, + parent: StagesTab, killEnabled: Boolean = false) { private val listener = parent.listener @@ -187,7 +187,7 @@ private[ui] class StageTableBase( private[ui] class FailedStageTable( stages: Seq[StageInfo], - parent: JobProgressTab, + parent: StagesTab, killEnabled: Boolean = false) extends StageTableBase(stages, parent, killEnabled) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala similarity index 91% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 03ca918e2e8b..70f07feca4e6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -23,14 +23,14 @@ import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.{SparkUI, SparkUITab} -/** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "stages") { +/** Web UI showing progress status of all stages in the given SparkContext. */ +private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { val sc = parent.sc val conf = sc.map(_.conf).getOrElse(new SparkConf) val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) val listener = parent.jobProgressListener - attachPage(new JobProgressPage(this)) + attachPage(new AllStagesPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) From 4487dcbe9df550ee054f2ba546ce0553ae40d103 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 29 Oct 2014 17:53:51 -0700 Subject: [PATCH 02/35] [SPARK-4145] Web UI job pages This PR adds two new pages to the Spark Web UI: - A jobs overview page, which shows details on running / completed / failed jobs. - A job details page, which displays information on an individual job's stages. The jobs overview page is now the default UI homepage; the old homepage is still accessible at /stages. In some situations, the jobs page may display "No information on stage" for some stages of pending jobs that have not begun executing. This is due to some limitations in how JobProgressListener finds out about stages. We can address this later as part of a separate scheduler PR. --- .../scala/org/apache/spark/ui/SparkUI.scala | 13 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 139 ++++++++++++++++++ .../apache/spark/ui/jobs/AllStagesPage.scala | 10 +- .../org/apache/spark/ui/jobs/JobPage.scala | 101 +++++++++++++ .../spark/ui/jobs/JobProgressListener.scala | 9 +- .../org/apache/spark/ui/jobs/JobsTab.scala | 30 ++++ .../org/apache/spark/ui/jobs/PoolPage.scala | 4 +- .../org/apache/spark/ui/jobs/StageTable.scala | 21 +-- .../org/apache/spark/ui/jobs/StagesTab.scala | 4 +- .../org/apache/spark/ui/jobs/UIData.scala | 2 + 10 files changed, 307 insertions(+), 26 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 9427ee9069c0..a96305e44c4c 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -23,7 +23,7 @@ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} -import org.apache.spark.ui.jobs.{JobProgressListener, StagesTab} +import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} /** @@ -45,20 +45,23 @@ private[spark] class SparkUI private ( /** Initialize all components of the server. */ def initialize() { - val jobProgressTab = new StagesTab(this) - attachTab(jobProgressTab) + attachTab(new JobsTab(this)) + val stagesTab = new StagesTab(this) + attachTab(stagesTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) + attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) attachHandler( - createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) + createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) // If the UI is live, then serve sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) } } initialize() + val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + def getAppName = appName /** Set the app name for this UI. */ 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 new file mode 100644 index 000000000000..0a81bc60f1bb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -0,0 +1,139 @@ +/* + * 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.jobs + +import javax.servlet.http.HttpServletRequest + +import scala.xml.{Node, NodeSeq} + +import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.jobs.UIData.JobUIData + + +/** Page showing list of all ongoing and recently finished stages and pools */ +private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { + private val sc = parent.sc + private val listener = parent.listener + + private def getSubmissionTime(job: JobUIData): Option[Long] = { + for ( + firstStageId <- job.stageIds.headOption; + firstStageInfo <- listener.stageIdToInfo.get(firstStageId); + submitTime <- firstStageInfo.submissionTime + ) yield submitTime + } + + private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = { + val columns: Seq[Node] = { + Job Id (Job Group) + Description + Submitted + Duration + } + + def makeRow(job: JobUIData): Seq[Node] = { + val lastStageInfo = job.stageIds.lastOption.flatMap(listener.stageIdToInfo.get) + val lastStageData = lastStageInfo.flatMap { s => + listener.stageIdToData.get((s.stageId, s.attemptId)) + } + val duration: Option[Long] = { + job.startTime.map { start => + val end = job.endTime.getOrElse(System.currentTimeMillis()) + end - start + } + } + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") + val formattedSubmissionTime = job.startTime.map(UIUtils.formatDate).getOrElse("Unknown") + val detailUrl = + "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) + + + + {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} + + +
{lastStageData.flatMap(_.description).getOrElse("")}
+ {lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)")} + + + {formattedSubmissionTime} + + {formattedDuration} + + } + + + {columns} + + {jobs.map(makeRow)} + +
+ } + + def render(request: HttpServletRequest): Seq[Node] = { + listener.synchronized { + val activeJobs = listener.activeJobs.values.toSeq + val completedJobs = listener.completedJobs.reverse.toSeq + val failedJobs = listener.failedJobs.reverse.toSeq + val now = System.currentTimeMillis + + val activeJobsTable = + jobsTable(activeJobs.sortBy(getSubmissionTime(_).getOrElse(-1L)).reverse) + val completedJobsTable = + jobsTable(completedJobs.sortBy(getSubmissionTime(_).getOrElse(-1L)).reverse) + val failedJobsTable = + jobsTable(failedJobs.sortBy(getSubmissionTime(_).getOrElse(-1L)).reverse) + + val summary: NodeSeq = +
+
    + {if (sc.isDefined) { + // Total duration is not meaningful unless the UI is live +
  • + Total Duration: + {UIUtils.formatDuration(now - sc.get.startTime)} +
  • + }} +
  • + Scheduling Mode: + {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} +
  • +
  • + Active Jobs: + {activeJobs.size} +
  • +
  • + Completed Jobs: + {completedJobs.size} +
  • +
  • + Failed Jobs: + {failedJobs.size} +
  • +
+
+ + val content = summary ++ +

Active Jobs ({activeJobs.size})

++ activeJobsTable ++ +

Completed Jobs ({completedJobs.size})

++ completedJobsTable ++ +

Failed Jobs ({failedJobs.size})

++ failedJobsTable + + UIUtils.headerSparkPage("Spark Jobs", content, parent) + } + } +} 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 3b7d55361e44..88d55257cdcf 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 @@ -39,11 +39,13 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent, parent.killEnabled) + parent.basePath, parent.listener, parent.killEnabled) val completedStagesTable = - new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent) + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, parent.killEnabled) val failedStagesTable = - new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, parent.killEnabled) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) @@ -91,7 +93,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {

Failed Stages ({failedStages.size})

++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage("Spark Stages", content, parent) + UIUtils.headerSparkPage("Spark Stages (for all jobs)", content, parent) } } } 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 new file mode 100644 index 000000000000..286ed6e2f639 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -0,0 +1,101 @@ +/* + * 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.jobs + +import javax.servlet.http.HttpServletRequest + +import org.apache.spark.scheduler.StageInfo + +import scala.xml.{NodeSeq, Node} + +import org.apache.spark.ui.{UIUtils, WebUIPage} + +/** Page showing statistics and stage list for a given job */ +private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { + private val listener = parent.listener + private val sc = parent.sc + + def render(request: HttpServletRequest): Seq[Node] = { + listener.synchronized { + val jobId = request.getParameter("id").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 stages = jobData.stageIds.map { stageId => + listener.stageIdToInfo.getOrElse(stageId, + new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) + } + + val (activeStages, completedOrFailedStages) = stages.partition(_.completionTime.isDefined) + val (failedStages, completedStages) = + completedOrFailedStages.partition(_.failureReason.isDefined) + + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, parent.killEnabled) + val completedStagesTable = + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, parent.killEnabled) + val failedStagesTable = + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, parent.killEnabled) + + val summary: NodeSeq = +
+
    + { + if (jobData.jobGroup.isDefined) { +
  • + Job Group: + {jobData.jobGroup.get} +
  • + } else Seq.empty + } +
  • + Active Stages: + {activeStages.size} +
  • +
  • + Completed Stages: + {completedStages.size} +
  • +
  • + Failed Stages: + {failedStages.size} +
  • +
+
+ + val content = summary ++ +

Active Stages ({activeStages.size})

++ + activeStagesTable.toNodeSeq ++ +

Completed Stages ({completedStages.size})

++ + completedStagesTable.toNodeSeq ++ +

Failed Stages ({failedStages.size})

++ + failedStagesTable.toNodeSeq + UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index b5207360510d..d94643d9427d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -70,9 +70,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { def blockManagerIds = executorIdToBlockManagerId.values.toSeq override def onJobStart(jobStart: SparkListenerJobStart) = synchronized { - val jobGroup = Option(jobStart.properties).map(_.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + val jobGroup = for ( + props <- Option(jobStart.properties); + group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + ) yield group val jobData: JobUIData = - new JobUIData(jobStart.jobId, jobStart.stageIds, jobGroup, JobExecutionStatus.RUNNING) + new JobUIData(jobStart.jobId, Some(System.currentTimeMillis), None, jobStart.stageIds, + jobGroup, JobExecutionStatus.RUNNING) jobIdToData(jobStart.jobId) = jobData activeJobs(jobStart.jobId) = jobData } @@ -82,6 +86,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { logWarning(s"Job completed for unknown job ${jobEnd.jobId}") new JobUIData(jobId = jobEnd.jobId) } + jobData.endTime = Some(System.currentTimeMillis()) jobEnd.jobResult match { case JobSucceeded => completedJobs += jobData 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 new file mode 100644 index 000000000000..e37f890ff0a8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.jobs + +import org.apache.spark.ui.{SparkUI, SparkUITab} + +/** Web UI showing progress status of all jobs in the given SparkContext. */ +private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { + val sc = parent.sc + val killEnabled = parent.killEnabled + val listener = parent.jobProgressListener + + attachPage(new AllJobsPage(this)) + attachPage(new JobPage(this)) +} 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 968aaba301c7..f944ee860db7 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 @@ -37,8 +37,8 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { case Some(s) => s.values.toSeq case None => Seq[StageInfo]() } - val activeStagesTable = - new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, parent.killEnabled) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getPoolForName(poolName).get).toSeq 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 63a163d3c2b0..1546b26c0fc2 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 @@ -29,12 +29,11 @@ import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTableBase( stages: Seq[StageInfo], - parent: StagesTab, + basePath: String, + listener: JobProgressListener, + isFairScheduler: Boolean, killEnabled: Boolean = false) { - private val listener = parent.listener - protected def isFairScheduler = parent.isFairScheduler - protected def columns: Seq[Node] = { Stage Id ++ {if (isFairScheduler) {Pool Name} else Seq.empty} ++ @@ -88,7 +87,7 @@ private[ui] class StageTableBase( // scalastyle:off val killLink = if (killEnabled) { val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId) + .format(UIUtils.prependBaseUri(basePath), s.stageId) val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');" .format(s.stageId) @@ -98,7 +97,7 @@ private[ui] class StageTableBase( // scalastyle:on val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId, s.attemptId) + .format(UIUtils.prependBaseUri(basePath), s.stageId, s.attemptId) val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -112,7 +111,7 @@ private[ui] class StageTableBase( Text("RDD: ") ++ // scalastyle:off cachedRddInfos.map { i => - {i.name} + {i.name} } // scalastyle:on }} @@ -162,7 +161,7 @@ private[ui] class StageTableBase( {if (isFairScheduler) { + .format(UIUtils.prependBaseUri(basePath), stageData.schedulingPool)}> {stageData.schedulingPool} @@ -187,9 +186,11 @@ private[ui] class StageTableBase( private[ui] class FailedStageTable( stages: Seq[StageInfo], - parent: StagesTab, + basePath: String, + listener: JobProgressListener, + isFairScheduler: Boolean, killEnabled: Boolean = false) - extends StageTableBase(stages, parent, killEnabled) { + extends StageTableBase(stages, basePath, listener, isFairScheduler, killEnabled) { override protected def columns: Seq[Node] = super.columns ++ Failure Reason 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 70f07feca4e6..937261de00e3 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 @@ -19,15 +19,13 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.{SparkUI, SparkUITab} /** Web UI showing progress status of all stages in the given SparkContext. */ private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { val sc = parent.sc - val conf = sc.map(_.conf).getOrElse(new SparkConf) - val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val killEnabled = parent.killEnabled val listener = parent.jobProgressListener attachPage(new AllStagesPage(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index e2813f8eb5ab..7ab4bf4712a4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -39,6 +39,8 @@ private[jobs] object UIData { class JobUIData( var jobId: Int = -1, + var startTime: Option[Long] = None, + var endTime: Option[Long] = None, var stageIds: Seq[Int] = Seq.empty, var jobGroup: Option[String] = None, var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN From bfce2b9b11abb00a272804cdc5ea24ed8923712b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 5 Nov 2014 16:12:50 -0800 Subject: [PATCH 03/35] Address review comments, except for progress bar. --- .../org/apache/spark/ui/jobs/AllJobsPage.scala | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 0a81bc60f1bb..170ee5af56f5 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 @@ -17,15 +17,15 @@ package org.apache.spark.ui.jobs -import javax.servlet.http.HttpServletRequest - import scala.xml.{Node, NodeSeq} +import javax.servlet.http.HttpServletRequest + import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData.JobUIData -/** Page showing list of all ongoing and recently finished stages and pools */ +/** Page showing list of all ongoing and recently finished jobs */ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { private val sc = parent.sc private val listener = parent.listener @@ -103,12 +103,12 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
    {if (sc.isDefined) { - // Total duration is not meaningful unless the UI is live -
  • - Total Duration: - {UIUtils.formatDuration(now - sc.get.startTime)} -
  • - }} + // Total duration is not meaningful unless the UI is live +
  • + Total Duration: + {UIUtils.formatDuration(now - sc.get.startTime)} +
  • + }}
  • Scheduling Mode: {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} From 45343b8c966ec41d0e45091bdb2f74cd9e7268b7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 5 Nov 2014 16:21:25 -0800 Subject: [PATCH 04/35] More comments --- core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 286ed6e2f639..e2db51449e56 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,12 +17,11 @@ package org.apache.spark.ui.jobs +import scala.xml.{NodeSeq, Node} + import javax.servlet.http.HttpServletRequest import org.apache.spark.scheduler.StageInfo - -import scala.xml.{NodeSeq, Node} - import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing statistics and stage list for a given job */ @@ -44,6 +43,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { } val jobData = jobDataOption.get 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, "Unknown")) } From a475ea15561d30f0b575c0a2f3c06131f73e3818 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 10 Nov 2014 23:43:17 -0800 Subject: [PATCH 05/35] Add progress bars to jobs page. --- .../scala/org/apache/spark/ui/UIUtils.scala | 13 ++++++ .../apache/spark/ui/jobs/AllJobsPage.scala | 5 +++ .../spark/ui/jobs/JobProgressListener.scala | 45 ++++++++++++++++++- .../org/apache/spark/ui/jobs/StageTable.scala | 16 +------ .../org/apache/spark/ui/jobs/UIData.scala | 6 ++- 5 files changed, 67 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 3312671b6f88..5b9db86c03aa 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -283,4 +283,17 @@ private[spark] object UIUtils extends Logging { } + + 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) + +
    + + {completed}/{total} { if (failed > 0) s"($failed failed)" else "" } + +
    +
    +
    + } } 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 170ee5af56f5..c116d063714d 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 @@ -44,6 +44,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { Description Submitted Duration + Tasks: Succeeded/Total } def makeRow(job: JobUIData): Seq[Node] = { @@ -74,6 +75,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {formattedSubmissionTime} {formattedDuration} + + {UIUtils.makeProgressBar(job.numActiveTasks, job.numCompletedTasks, + job.numFailedTasks, job.numTasks)} + } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 72bc8d08864a..d63722ceec36 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -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 @@ -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 @@ -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 { @@ -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 { @@ -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) } @@ -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 { @@ -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) { @@ -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 => @@ -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 + } + } } } 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 1546b26c0fc2..88b687d51a53 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 @@ -69,20 +69,6 @@ private[ui] class StageTableBase( } - 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) - -
    - - {completed}/{total} { if (failed > 0) s"($failed failed)" else "" } - -
    -
    -
    - } - private def makeDescription(s: StageInfo): Seq[Node] = { // scalastyle:off val killLink = if (killEnabled) { @@ -172,7 +158,7 @@ private[ui] class StageTableBase( {submissionTime} {formattedDuration} - {makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size, + {UIUtils.makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size, stageData.numFailedTasks, s.numTasks)} {inputReadWithUnit} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 7ab4bf4712a4..829500e254db 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -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 { From 56701fa9925e01d9d9ae04bfcf283cf28866cbe3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 11 Nov 2014 14:10:40 -0800 Subject: [PATCH 06/35] Move last stage name / description logic out of markup. --- .../main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index c116d063714d..9e69b94f516f 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 @@ -52,6 +52,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { 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).getOrElse("") val duration: Option[Long] = { job.startTime.map { start => val end = job.endTime.getOrElse(System.currentTimeMillis()) @@ -68,8 +70,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} -
    {lastStageData.flatMap(_.description).getOrElse("")}
    - {lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)")} +
    {lastStageDescription}
    + {lastStageName} {formattedSubmissionTime} From 1cf4987f65f9c5e017d70675518ebbba6c35eaf0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 11 Nov 2014 15:19:35 -0800 Subject: [PATCH 07/35] Fix broken kill links; add Selenium test to avoid future regressions. --- .../scala/org/apache/spark/ui/SparkUI.scala | 4 +- .../apache/spark/ui/jobs/AllStagesPage.scala | 8 ++-- .../org/apache/spark/ui/jobs/JobPage.scala | 8 ++-- .../org/apache/spark/ui/jobs/JobsTab.scala | 2 + .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StageTable.scala | 7 ++-- .../org/apache/spark/ui/UISeleniumSuite.scala | 41 +++++++++++++++++-- 7 files changed, 56 insertions(+), 17 deletions(-) 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 a96305e44c4c..176907dffa46 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -43,6 +43,8 @@ private[spark] class SparkUI private ( extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") with Logging { + val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + /** Initialize all components of the server. */ def initialize() { attachTab(new JobsTab(this)) @@ -60,8 +62,6 @@ private[spark] class SparkUI private ( } initialize() - val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) - def getAppName = appName /** Set the app name for this UI. */ 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 1216b1783788..908c96dfae49 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 @@ -41,13 +41,15 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.listener, parent.killEnabled) + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) val completedStagesTable = new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, parent.killEnabled) + parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) val failedStagesTable = new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, parent.killEnabled) + parent.listener, isFairScheduler = parent.isFairScheduler) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) 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 e2db51449e56..b5da390ef485 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 @@ -55,13 +55,15 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.listener, parent.killEnabled) + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) val completedStagesTable = new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, parent.killEnabled) + parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) val failedStagesTable = new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, parent.killEnabled) + parent.listener, isFairScheduler = parent.isFairScheduler) val summary: NodeSeq =
    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 e37f890ff0a8..b2bbfdee5694 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 @@ -17,12 +17,14 @@ package org.apache.spark.ui.jobs +import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.{SparkUI, SparkUITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { val sc = parent.sc val killEnabled = parent.killEnabled + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) val listener = parent.jobProgressListener attachPage(new AllJobsPage(this)) 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 f944ee860db7..5fc6cc753315 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 @@ -38,7 +38,8 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { case None => Seq[StageInfo]() } val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent.basePath, parent.listener, parent.killEnabled) + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getPoolForName(poolName).get).toSeq 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 88b687d51a53..4b692fb4de96 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 @@ -32,7 +32,7 @@ private[ui] class StageTableBase( basePath: String, listener: JobProgressListener, isFairScheduler: Boolean, - killEnabled: Boolean = false) { + killEnabled: Boolean) { protected def columns: Seq[Node] = { Stage Id ++ @@ -174,9 +174,8 @@ private[ui] class FailedStageTable( stages: Seq[StageInfo], basePath: String, listener: JobProgressListener, - isFairScheduler: Boolean, - killEnabled: Boolean = false) - extends StageTableBase(stages, basePath, listener, isFairScheduler, killEnabled) { + isFairScheduler: Boolean) + extends StageTableBase(stages, basePath, listener, isFairScheduler, killEnabled = false) { override protected def columns: Seq[Node] = super.columns ++ Failure Reason diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index bacf6a16fc23..8c5fcc370fab 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.ui -import org.apache.spark.api.java.StorageLevels -import org.apache.spark.{SparkException, SparkConf, SparkContext} import org.openqa.selenium.WebDriver import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest._ @@ -26,6 +24,9 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.{SparkException, SparkConf, SparkContext} +import org.apache.spark.SparkContext._ import org.apache.spark.LocalSparkContext._ /** @@ -89,7 +90,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to sc.ui.get.appUIAddress + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")).get.text should be("Active Stages (0)") find(id("failed")).get.text should be("Failed Stages (1)") } @@ -101,7 +102,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { sc.parallelize(1 to 10).map { x => unserializableObject}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to sc.ui.get.appUIAddress + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")).get.text should be("Active Stages (0)") // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: @@ -109,4 +110,36 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + test("spark.ui.killEnabled should properly control kill button display") { + def getSparkContext(killEnabled: Boolean): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + .set("spark.ui.killEnabled", killEnabled.toString) + new SparkContext(conf) + } + + def hasKillLink = find(className("kill-link")).isDefined + def runSlowJob(sc: SparkContext) { + sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() + } + + withSpark(getSparkContext(killEnabled = true)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + assert(hasKillLink) + } + } + + withSpark(getSparkContext(killEnabled = false)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + assert(!hasKillLink) + } + } + } } From 85e9c852e497304b9a5442845ee9dafe3b673401 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 11 Nov 2014 15:22:43 -0800 Subject: [PATCH 08/35] Extract startTime into separate variable. --- .../main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 9e69b94f516f..85b318051858 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 @@ -27,7 +27,7 @@ import org.apache.spark.ui.jobs.UIData.JobUIData /** Page showing list of all ongoing and recently finished jobs */ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { - private val sc = parent.sc + private val startTime: Option[Long] = parent.sc.map(_.startTime) private val listener = parent.listener private def getSubmissionTime(job: JobUIData): Option[Long] = { @@ -109,11 +109,11 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val summary: NodeSeq =
      - {if (sc.isDefined) { + {if (startTime.isDefined) { // Total duration is not meaningful unless the UI is live
    • Total Duration: - {UIUtils.formatDuration(now - sc.get.startTime)} + {UIUtils.formatDuration(now - startTime.get)}
    • }}
    • From 4d58e55faeaf76b49281c0e2f4c078011a281199 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 11 Nov 2014 15:23:39 -0800 Subject: [PATCH 09/35] Change label to "Tasks (for all stages)" --- core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 85b318051858..5d9889194ac7 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 @@ -44,7 +44,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { Description Submitted Duration - Tasks: Succeeded/Total + Tasks (for all stages): Succeeded/Total } def makeRow(job: JobUIData): Seq[Node] = { From 4846ce4414fff1e7bf8dda43d3025e8718287f78 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 11 Nov 2014 17:39:09 -0800 Subject: [PATCH 10/35] Hide "(Job Group") if no jobs were submitted in job groups. --- .../apache/spark/ui/jobs/AllJobsPage.scala | 4 +++- .../org/apache/spark/ui/UISeleniumSuite.scala | 21 +++++++++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) 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 5d9889194ac7..9c094bdcfc42 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 @@ -39,8 +39,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = { + val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) + val columns: Seq[Node] = { - Job Id (Job Group) + {if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id"} Description Submitted Duration diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 8c5fcc370fab..1ad95cc92529 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -142,4 +142,25 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + test("jobs page should not display job group name unless some job was submitted in a job group") { + withSpark(newSparkContext()) { sc => + val ui = sc.ui.get + // If no job has been run in a job group, then "(Job Group)" should not appear in the header + sc.parallelize(Seq(1, 2, 3)).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq + tableHeaders should not contain "Job Id (Job Group)" + } + // Once at least one job has been run in a job group, then we should display the group name: + sc.setJobGroup("my-job-group", "my-job-group-description") + sc.parallelize(Seq(1, 2, 3)).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq + tableHeaders should contain ("Job Id (Job Group)") + } + } + } } From b7bf30e6e2a9633015b18f3d93c5d86fe473fb50 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 11 Nov 2014 17:54:40 -0800 Subject: [PATCH 11/35] Add stages progress bar; fix bug where active stages show as completed. --- .../org/apache/spark/ui/jobs/AllJobsPage.scala | 7 ++++++- .../scala/org/apache/spark/ui/jobs/JobPage.scala | 2 +- .../apache/spark/ui/jobs/JobProgressListener.scala | 14 ++++++++++++++ .../scala/org/apache/spark/ui/jobs/UIData.scala | 7 ++++++- 4 files changed, 27 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 9c094bdcfc42..29a83c2ce8ad 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 @@ -46,7 +46,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { Description Submitted Duration - Tasks (for all stages): Succeeded/Total + Stages: Succeeded/Total + Tasks (for all stages): Succeeded/Total } def makeRow(job: JobUIData): Seq[Node] = { @@ -79,6 +80,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {formattedSubmissionTime} {formattedDuration} + + {UIUtils.makeProgressBar(job.numActiveStages, job.numCompletedStages, + job.numFailedStages, job.stageIds.size)} + {UIUtils.makeProgressBar(job.numActiveTasks, job.numCompletedTasks, job.numFailedTasks, job.numTasks)} 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 b5da390ef485..72634369c13d 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 @@ -49,7 +49,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) } - val (activeStages, completedOrFailedStages) = stages.partition(_.completionTime.isDefined) + val (completedOrFailedStages, activeStages) = stages.partition(_.completionTime.isDefined) val (failedStages, completedStages) = completedOrFailedStages.partition(_.failureReason.isDefined) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index d63722ceec36..b51e92c6d348 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -136,6 +136,19 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { numFailedStages += 1 trimIfNecessary(failedStages) } + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveStages -= 1 + if (stage.failureReason.isEmpty) { + jobData.numCompletedStages += 1 + } else { + jobData.numFailedStages += 1 + } + } } /** If stages is too large, remove and garbage collect old stages */ @@ -177,6 +190,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { jobData <- jobIdToData.get(jobId) ) { jobData.numTasks += stage.numTasks + jobData.numActiveStages += 1 } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 829500e254db..da67b3fa5f68 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -44,10 +44,15 @@ private[jobs] object UIData { var stageIds: Seq[Int] = Seq.empty, var jobGroup: Option[String] = None, var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, + /* Tasks */ var numTasks: Int = 0, var numActiveTasks: Int = 0, var numCompletedTasks: Int = 0, - var numFailedTasks: Int = 0 + var numFailedTasks: Int = 0, + /* Stages */ + var numActiveStages: Int = 0, + var numCompletedStages: Int = 0, + var numFailedStages: Int = 0 ) class StageUIData { From 8a2351bd51a2703efe950148779f9c03bc67fd3d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 11 Nov 2014 18:13:30 -0800 Subject: [PATCH 12/35] Add help tooltip to Spark Jobs page. --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 7 ++++++- .../main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 6 +++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 5b9db86c03aa..e9ad0013cfbd 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -169,7 +169,8 @@ private[spark] object UIUtils extends Logging { title: String, content: => Seq[Node], activeTab: SparkUITab, - refreshInterval: Option[Int] = None): Seq[Node] = { + refreshInterval: Option[Int] = None, + helpText: Option[String] = None): Seq[Node] = { val appName = activeTab.appName val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..." @@ -178,6 +179,9 @@ private[spark] object UIUtils extends Logging { {tab.name}
    • } + val helpButton: Seq[Node] = helpText.map { helpText => + (?) + }.getOrElse(Seq.empty) @@ -201,6 +205,7 @@ private[spark] object UIUtils extends Logging {

      {title} + {helpButton}

    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 29a83c2ce8ad..8c1da125dbac 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 @@ -147,7 +147,11 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {

    Completed Jobs ({completedJobs.size})

    ++ completedJobsTable ++

    Failed Jobs ({failedJobs.size})

    ++ failedJobsTable - UIUtils.headerSparkPage("Spark Jobs", content, parent) + val helpText = """A job is triggered by a action, like "count()" or "saveAsTextFile()".""" + + " Click on a job's title to see information about the stages of tasks associated with" + + " the job." + + UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) } } } From 1145c60c27d0e97e438070539d651755777ded50 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 17 Nov 2014 14:49:13 -0800 Subject: [PATCH 13/35] Display text instead of progress bar for stages. --- .../org/apache/spark/ui/jobs/AllJobsPage.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 8c1da125dbac..afd120f14a34 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 @@ -80,9 +80,17 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {formattedSubmissionTime} {formattedDuration} - - {UIUtils.makeProgressBar(job.numActiveStages, job.numCompletedStages, - job.numFailedStages, job.stageIds.size)} + + {job.numCompletedStages} + / + {job.stageIds.size} + { + if (job.numFailedStages > 0) { + ({job.numFailedStages} failed) + } else { + Seq.empty + } + } {UIUtils.makeProgressBar(job.numActiveTasks, job.numCompletedTasks, From d62ea7b652b91fa4b00ffddabc8eafc23e79a7d6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 17 Nov 2014 14:49:50 -0800 Subject: [PATCH 14/35] Add failing Selenium test for stage overcounting issue. --- .../org/apache/spark/ui/UISeleniumSuite.scala | 34 +++++++++++++++++-- 1 file changed, 32 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 1ad95cc92529..7f3a3a27e0c4 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -25,9 +25,10 @@ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ import org.apache.spark.api.java.StorageLevels -import org.apache.spark.{SparkException, SparkConf, SparkContext} +import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.shuffle.FetchFailedException /** * Selenium tests for the Spark Web UI. These tests are not run by default @@ -145,7 +146,6 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { test("jobs page should not display job group name unless some job was submitted in a job group") { withSpark(newSparkContext()) { sc => - val ui = sc.ui.get // If no job has been run in a job group, then "(Job Group)" should not appear in the header sc.parallelize(Seq(1, 2, 3)).count() eventually(timeout(5 seconds), interval(50 milliseconds)) { @@ -163,4 +163,34 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + test("stage failures / recomputations should not cause stages to be overcounted on job page") { + withSpark(newSparkContext()) { sc => + val data = sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity) + val shuffleHandle = + data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle + // Simulate fetch failures: + val mappedData = data.map { x => + val taskContext = TaskContext.get + if (taskContext.attemptId() == 1) { // Cause this stage to fail on its first attempt. + val env = SparkEnv.get + val bmAddress = env.blockManager.blockManagerId + val shuffleId = shuffleHandle.shuffleId + val mapId = 0 + val reduceId = taskContext.partitionId() + val message = "Simulated fetch failure" + throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message) + } else { + x + } + } + mappedData.count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + find(cssSelector(".stage-progress-cell .completed-stages")).get.text should be ("2") + find(cssSelector(".stage-progress-cell .total-stages")).get.text should be ("2") + find(cssSelector(".stage-progress-cell .failed-stages")).get.text should be ("(1 failed)") + } + } + } } From 79793cd6f3f4c85226e22684a821377fa88c5c6a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 13:54:03 -0800 Subject: [PATCH 15/35] Track indices of completed stage to avoid overcounting when failures occur. Remove spaces around slash in stage progress cell. --- .../scala/org/apache/spark/ui/jobs/AllJobsPage.scala | 12 ++---------- .../apache/spark/ui/jobs/JobProgressListener.scala | 2 +- .../main/scala/org/apache/spark/ui/jobs/UIData.scala | 2 +- .../scala/org/apache/spark/ui/UISeleniumSuite.scala | 4 +--- 4 files changed, 5 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index afd120f14a34..f80f50200738 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 @@ -81,16 +81,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {formattedDuration} - {job.numCompletedStages} - / - {job.stageIds.size} - { - if (job.numFailedStages > 0) { - ({job.numFailedStages} failed) - } else { - Seq.empty - } - } + {job.completedStageIndices.size}/{job.stageIds.size} + {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)" else ""} {UIUtils.makeProgressBar(job.numActiveTasks, job.numCompletedTasks, diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 01ab885df266..ccdae53b648d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -144,7 +144,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { ) { jobData.numActiveStages -= 1 if (stage.failureReason.isEmpty) { - jobData.numCompletedStages += 1 + jobData.completedStageIndices.add(stage.stageId) } else { jobData.numFailedStages += 1 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 3b3d83a7bd31..9694443cd795 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -52,7 +52,7 @@ private[jobs] object UIData { var numFailedTasks: Int = 0, /* Stages */ var numActiveStages: Int = 0, - var numCompletedStages: Int = 0, + var completedStageIndices: OpenHashSet[Int] = new OpenHashSet[Int](), var numFailedStages: Int = 0 ) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 7f3a3a27e0c4..d8e66fd08180 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -187,9 +187,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { mappedData.count() eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") - find(cssSelector(".stage-progress-cell .completed-stages")).get.text should be ("2") - find(cssSelector(".stage-progress-cell .total-stages")).get.text should be ("2") - find(cssSelector(".stage-progress-cell .failed-stages")).get.text should be ("(1 failed)") + find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") } } } From 5884f918f7271f1cd6eca38abefe408bc63c0a68 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 14:41:25 -0800 Subject: [PATCH 16/35] Add StageInfos to SparkListenerJobStart event. --- .../apache/spark/scheduler/DAGScheduler.scala | 7 +- .../spark/scheduler/SparkListener.scala | 6 +- .../org/apache/spark/util/JsonProtocol.scala | 5 +- .../apache/spark/util/JsonProtocolSuite.scala | 118 +++++++++++++++++- 4 files changed, 130 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 22449517d100..94d11178ade4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -751,14 +751,15 @@ class DAGScheduler( localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Seq.empty, Seq.empty, properties)) runLocally(job) } else { jobIdToActiveJob(jobId) = job activeJobs += job finalStage.resultOfJob = Some(job) - listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, - properties)) + val stageIds = jobIdToStageIds(jobId).toArray + val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) + listenerBus.post(SparkListenerJobStart(job.jobId, stageInfos, stageIds, properties)) submitStage(finalStage) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 86afe3bd5265..6b46a94d9fc9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -56,7 +56,11 @@ case class SparkListenerTaskEnd( extends SparkListenerEvent @DeveloperApi -case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) +case class SparkListenerJobStart( + jobId: Int, + stageInfos: Seq[StageInfo], + stageIds: Seq[Int], // Note: this is here for backwards-compatibility + properties: Properties = null) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 7e536edfe807..16dec5f51a86 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -121,6 +121,7 @@ private[spark] object JsonProtocol { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ + ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ ("Stage IDs" -> jobStart.stageIds) ~ ("Properties" -> properties) } @@ -455,7 +456,9 @@ private[spark] object JsonProtocol { val jobId = (json \ "Job ID").extract[Int] val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") - SparkListenerJobStart(jobId, stageIds, properties) + val stageInfos = Utils.jsonOption(json \ "Stage Infos") + .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse(Seq.empty) + SparkListenerJobStart(jobId, stageInfos, stageIds, properties) } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 50f42054b929..d2510ec8092a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -47,7 +47,12 @@ class JsonProtocolSuite extends FunSuite { val taskEndWithOutput = SparkListenerTaskEnd(1, 0, "ResultTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)) - val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) + val jobStart = { + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => + makeStageInfo(x * 100, x * 200, x * 300, x * 400L, x * 500L)) + SparkListenerJobStart(10, stageInfos, stageIds, properties) + } val jobEnd = SparkListenerJobEnd(20, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), @@ -224,6 +229,19 @@ class JsonProtocolSuite extends FunSuite { assert(expectedExecutorLostFailure === JsonProtocol.taskEndReasonFromJson(oldEvent)) } + test("SparkListenerJobStart backward compatibility") { + // Prior to Spark 1.2.0, SparkListenerJobStart did not have a "Stage Infos" property. + val jobStart = { + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => + makeStageInfo(x * 100, x * 200, x * 300, x * 400L, x * 500L)) + SparkListenerJobStart(10, stageInfos, stageIds, properties) + } + val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) + val expectedJobStart = SparkListenerJobStart(10, Seq.empty, Seq[Int](1, 2, 3, 4), properties) + assert(expectedJobStart === JsonProtocol.jobStartFromJson(oldEvent)) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ @@ -1051,6 +1069,104 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerJobStart", | "Job ID": 10, + | "Stage Infos": [ + | { + | "Stage ID": 100, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 200, + | "RDD Info": [ + | + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 200, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 400, + | "RDD Info": [ + | + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 300, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 600, + | "RDD Info": [ + | + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 400, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 800, + | "RDD Info": [ + | + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | } + | ], | "Stage IDs": [ | 1, | 2, From 8ab6c28c0049226ec187b4e8c938f9114c016d07 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 15:49:06 -0800 Subject: [PATCH 17/35] Compute numTasks from job start stage infos. --- .../apache/spark/ui/jobs/JobProgressListener.scala | 13 +++++++------ .../scala/org/apache/spark/ui/jobs/UIData.scala | 2 ++ .../scala/org/apache/spark/ui/UISeleniumSuite.scala | 6 ++++++ 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ccdae53b648d..7130daafe511 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -85,6 +85,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val jobData: JobUIData = new JobUIData(jobStart.jobId, Some(System.currentTimeMillis), None, jobStart.stageIds, jobGroup, JobExecutionStatus.RUNNING) + // Compute (a potential underestimate of) the number of tasks that will be run by this job: + jobData.numTasks = { + val allStages = jobStart.stageInfos + val missingStages = allStages.filter(_.completionTime.isEmpty) + missingStages.map(_.numTasks).sum + } jobIdToData(jobStart.jobId) = jobData activeJobs(jobStart.jobId) = jobData for (stageId <- jobStart.stageIds) { @@ -189,7 +195,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { jobId <- activeJobsDependentOnStage; jobData <- jobIdToData.get(jobId) ) { - jobData.numTasks += stage.numTasks jobData.numActiveStages += 1 } } @@ -245,8 +250,6 @@ 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 => @@ -279,9 +282,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { jobData.numActiveTasks -= 1 taskEnd.reason match { case Success => - if (!isRecomputation) { - jobData.numCompletedTasks += 1 - } + jobData.numCompletedTasks += 1 case _ => jobData.numFailedTasks += 1 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 9694443cd795..f849792da455 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -46,6 +46,8 @@ private[jobs] object UIData { var jobGroup: Option[String] = None, var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, /* Tasks */ + // `numTasks` is a potential underestimate of the true number of tasks that this job will run + // see https://github.com/apache/spark/pull/3009 for an extensive discussion of this var numTasks: Int = 0, var numActiveTasks: Int = 0, var numCompletedTasks: Int = 0, diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index d8e66fd08180..b18c6525d606 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -188,6 +188,12 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") + // Ideally, the following test would pass, but currently we over-count completed tasks + // if task recomputations occur: + // find(cssSelector(".progress-cell .progress")).get.text should be ("2/2 (1 failed)") + // Instead, we guarantee that the total number of tasks is always correct, while the number + // of completed tasks may be higher: + find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)") } } } From 8955f4c8a64d29b11fce08a23101b84bf4152c3e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 18:08:13 -0800 Subject: [PATCH 18/35] Display information for pending stages on jobs page. We still need to distinguish pending stages from active ones; I'll do this in a separate commit. --- .../spark/ui/jobs/JobProgressListener.scala | 6 ++++ .../org/apache/spark/ui/UISeleniumSuite.scala | 30 +++++++++++++++++-- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 7130daafe511..be81ca6ab9be 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -96,6 +96,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { for (stageId <- jobStart.stageIds) { stageIdToActiveJobIds.getOrElseUpdate(stageId, new HashSet[StageId]).add(jobStart.jobId) } + // If there's no information for a stage, store the StageInfo received from the scheduler + // so that we can display stage descriptions for pending stages: + for (stageInfo <- jobStart.stageInfos) { + stageIdToInfo.getOrElseUpdate(stageInfo.stageId, stageInfo) + stageIdToData.getOrElseUpdate((stageInfo.stageId, stageInfo.attemptId), new StageUIData) + } } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index b18c6525d606..fc2c83914763 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui -import org.openqa.selenium.WebDriver +import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest._ import org.scalatest.concurrent.Eventually._ @@ -164,7 +164,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } - test("stage failures / recomputations should not cause stages to be overcounted on job page") { + test("job progress bars should handle stage / task failures") { withSpark(newSparkContext()) { sc => val data = sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity) val shuffleHandle = @@ -188,7 +188,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") - // Ideally, the following test would pass, but currently we over-count completed tasks + // Ideally, the following test would pass, but currently we overcount completed tasks // if task recomputations occur: // find(cssSelector(".progress-cell .progress")).get.text should be ("2/2 (1 failed)") // Instead, we guarantee that the total number of tasks is always correct, while the number @@ -197,4 +197,28 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + test("job details page should display useful information for stages that haven't started") { + withSpark(newSparkContext()) { sc => + // Create a multi-stage job with a long delay in the first stage: + val rdd = sc.parallelize(Seq(1, 2, 3)).map { x => + // This long sleep call won't slow down the tests because we don't actually need to wait + // for the job to finish. + Thread.sleep(20000) + }.groupBy(identity).map(identity).groupBy(identity).map(identity) + // Start the job: + rdd.countAsync() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0") + // Essentially, we want to check that none of the stage rows show + // "No data available for this stage". Checking for the absence of that string is brittle + // because someone could change the error message and cause this test to pass by accident. + // Instead, it's safer to check that each row contains a link to a stage details page. + findAll(cssSelector("tbody tr")).foreach { row => + val link = row.underlying.findElement(By.xpath(".//a")) + link.getAttribute("href") should include ("stage") + } + } + } + } } From e2f2c4380619a9b222ae3cb68f5a8aee385ffdba Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 18:15:18 -0800 Subject: [PATCH 19/35] Fix sorting of stages in job details page. --- core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 72634369c13d..5c33551f1ce5 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 @@ -49,7 +49,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) } - val (completedOrFailedStages, activeStages) = stages.partition(_.completionTime.isDefined) + val (completedOrFailedStages, activeStages) = + stages.sortBy(_.stageId).reverse.partition(_.completionTime.isDefined) val (failedStages, completedStages) = completedOrFailedStages.partition(_.failureReason.isDefined) From 171b53cfc835d3f7fb85130a14e55b807689293f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 18:58:24 -0800 Subject: [PATCH 20/35] Move `startTime` to the start of SparkContext. This fixes a rare race-condition where someone could access a UI page before `SparkContext.startTime` was initialized, causing the UI to display an incorrect total duration. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 65edeeffb837..6b8c4eb9afe3 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -69,6 +69,8 @@ class SparkContext(config: SparkConf) extends Logging { // contains a map from hostname to a list of input format splits on the host. private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + val startTime = System.currentTimeMillis() + /** * Create a SparkContext that loads settings from system properties (for instance, when * launching with ./bin/spark-submit). @@ -248,8 +250,6 @@ class SparkContext(config: SparkConf) extends Logging { /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) - val startTime = System.currentTimeMillis() - // Add each JAR given through the constructor if (jars != null) { jars.foreach(addJar) From f2a15da357fe30c590b1836675808baadcbfb817 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 18:59:54 -0800 Subject: [PATCH 21/35] Add status field to job details page. --- core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala | 4 ++++ 1 file changed, 4 insertions(+) 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 5c33551f1ce5..32004d435eb4 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 @@ -69,6 +69,10 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val summary: NodeSeq =
      +
    • + Status: + {jobData.status} +
    • { if (jobData.jobGroup.isDefined) {
    • From 5eb39dcbdeb23ff9597fe7306865d3990ccb4bf5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 19:20:35 -0800 Subject: [PATCH 22/35] Add pending stages table to job page. --- .../org/apache/spark/ui/jobs/JobPage.scala | 32 ++++++++++++++++--- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 32004d435eb4..465d2698d155 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,6 +17,7 @@ package org.apache.spark.ui.jobs +import scala.collection.mutable import scala.xml.{NodeSeq, Node} import javax.servlet.http.HttpServletRequest @@ -49,15 +50,32 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) } - val (completedOrFailedStages, activeStages) = - stages.sortBy(_.stageId).reverse.partition(_.completionTime.isDefined) - val (failedStages, completedStages) = - completedOrFailedStages.partition(_.failureReason.isDefined) + val pendingStages = mutable.Buffer[StageInfo]() + val activeStages = mutable.Buffer[StageInfo]() + val completedStages = mutable.Buffer[StageInfo]() + val failedStages = mutable.Buffer[StageInfo]() + for (stage <- stages) { + if (stage.submissionTime.isEmpty) { + pendingStages += stage + } else if (stage.completionTime.isDefined) { + if (stage.failureReason.isDefined) { + failedStages += stage + } else { + completedStages += stage + } + } else { + activeStages += stage + } + } val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = parent.killEnabled) + val pendingStagesTable = + new StageTableBase(pendingStages.sortBy(_.stageId).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = false) val completedStagesTable = new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, @@ -85,6 +103,10 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { Active Stages: {activeStages.size}
    • +
    • + Pending Stages: + {pendingStages.size} +
    • Completed Stages: {completedStages.size} @@ -99,6 +121,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val content = summary ++

      Active Stages ({activeStages.size})

      ++ activeStagesTable.toNodeSeq ++ +

      Pending Stages ({pendingStages.size})

      ++ + pendingStagesTable.toNodeSeq ++

      Completed Stages ({completedStages.size})

      ++ completedStagesTable.toNodeSeq ++

      Failed Stages ({failedStages.size})

      ++ From d69c77586d5cea40fc5430d103c5f072ac8ba703 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 18 Nov 2014 19:33:51 -0800 Subject: [PATCH 23/35] Fix table sorting on all jobs page. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit It now sorts based on the time that the scheduler learns about the job rather than the time that the first stage begins executing. This is necessary because the earliest stage that a job depends on may not necessarily be executed (e.g. if it lies behind a shuffle boundary for a shuffle that’s already been computed, or if some RDD has been cached, etc.). --- .../org/apache/spark/ui/jobs/AllJobsPage.scala | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index f80f50200738..d093ea98bb82 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 @@ -30,14 +30,6 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { private val startTime: Option[Long] = parent.sc.map(_.startTime) private val listener = parent.listener - private def getSubmissionTime(job: JobUIData): Option[Long] = { - for ( - firstStageId <- job.stageIds.headOption; - firstStageInfo <- listener.stageIdToInfo.get(firstStageId); - submitTime <- firstStageInfo.submissionTime - ) yield submitTime - } - private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = { val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) @@ -107,11 +99,11 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val now = System.currentTimeMillis val activeJobsTable = - jobsTable(activeJobs.sortBy(getSubmissionTime(_).getOrElse(-1L)).reverse) + jobsTable(activeJobs.sortBy(_.startTime.getOrElse(-1L)).reverse) val completedJobsTable = - jobsTable(completedJobs.sortBy(getSubmissionTime(_).getOrElse(-1L)).reverse) + jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) val failedJobsTable = - jobsTable(failedJobs.sortBy(getSubmissionTime(_).getOrElse(-1L)).reverse) + jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) val summary: NodeSeq =
      From 67080ba5fa930cdfad0c6ba367e2264e9ed2c3b3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 19 Nov 2014 18:06:11 -0800 Subject: [PATCH 24/35] Ensure that "phantom stages" don't cause memory leaks. --- .../apache/spark/ui/jobs/JobProgressListener.scala | 14 +++++++++++++- .../spark/ui/jobs/JobProgressListenerSuite.scala | 3 ++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 6ba80dbc3d95..a8941fdb8e93 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -189,7 +189,19 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { jobData.status = JobExecutionStatus.FAILED } for (stageId <- jobData.stageIds) { - stageIdToActiveJobIds.get(stageId).foreach(_.remove(jobEnd.jobId)) + stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => + jobsUsingStage.remove(jobEnd.jobId) + stageIdToInfo.get(stageId).foreach { stageInfo => + // If this is a pending stage and no other job depends on it, then it won't be run. + // To prevent memory leaks, remove this data since it won't be cleaned up as stages + // finish / fail: + if (stageInfo.submissionTime.isEmpty && stageInfo.completionTime.isEmpty + && jobsUsingStage.isEmpty) { + stageIdToInfo.remove(stageId) + stageIdToData.remove((stageId, stageInfo.attemptId)) + } + } + } } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 19fcb80d1db5..53e7b9b4e613 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -55,8 +55,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc } private def runJob(listener: SparkListener, jobId: Int, shouldFail: Boolean = false) { + val stagesThatWontBeRun = jobId * 200 to jobId * 200 + 10 val stageIds = jobId * 100 to jobId * 100 + 50 - listener.onJobStart(createJobStartEvent(jobId, stageIds)) + listener.onJobStart(createJobStartEvent(jobId, stageIds ++ stagesThatWontBeRun)) for (stageId <- stageIds) { listener.onStageSubmitted(createStageStartEvent(stageId)) listener.onStageCompleted(createStageEndEvent(stageId, failed = stageId % 2 == 0)) From eebdc2cb016409ae17e06b126c7459b587f1c358 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 19 Nov 2014 18:21:16 -0800 Subject: [PATCH 25/35] =?UTF-8?q?Don=E2=80=99t=20display=20pending=20stage?= =?UTF-8?q?s=20for=20completed=20jobs.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../org/apache/spark/ui/jobs/JobPage.scala | 5 +++-- .../org/apache/spark/ui/UISeleniumSuite.scala | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 465d2698d155..c02277120ada 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 @@ -22,13 +22,13 @@ import scala.xml.{NodeSeq, Node} import javax.servlet.http.HttpServletRequest +import org.apache.spark.JobExecutionStatus import org.apache.spark.scheduler.StageInfo import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing statistics and stage list for a given job */ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { private val listener = parent.listener - private val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -43,6 +43,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { 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 @@ -55,7 +56,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val completedStages = mutable.Buffer[StageInfo]() val failedStages = mutable.Buffer[StageInfo]() for (stage <- stages) { - if (stage.submissionTime.isEmpty) { + if (!isComplete && stage.submissionTime.isEmpty) { pendingStages += stage } else if (stage.completionTime.isDefined) { if (stage.failureReason.isDefined) { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index fc2c83914763..b1ff39835ed2 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -221,4 +221,21 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + test("stages that aren't run do not show up in 'pending stages' after a job finishes") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + // Check that the "pending stages" section is empty: + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") + find(id("pending")).get.text should be ("Pending Stages (0)") + } + } + } } From 034aa8d0db7454fd04fc39600efddbf514c4150b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 19 Nov 2014 18:31:44 -0800 Subject: [PATCH 26/35] Use `.max()` to find result stage for job. --- .../apache/spark/ui/jobs/AllJobsPage.scala | 2 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 20 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) 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 d093ea98bb82..286736ff1b70 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 @@ -43,7 +43,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } def makeRow(job: JobUIData): Seq[Node] = { - val lastStageInfo = job.stageIds.lastOption.flatMap(listener.stageIdToInfo.get) + val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) val lastStageData = lastStageInfo.flatMap { s => listener.stageIdToData.get((s.stageId, s.attemptId)) } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index b1ff39835ed2..f116e4e821a2 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -238,4 +238,24 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + + test("jobs with stages thatare skipped should show correct link descriptions on jobs page") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + findAll(cssSelector("tbody tr a")).foreach { link => + link.text.toLowerCase should include ("count") + link.text.toLowerCase should not include "unknown" + } + } + } + } } From 0b77e3e14e9eafc523ac17cc70b4be4766ed0930 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 19 Nov 2014 18:36:22 -0800 Subject: [PATCH 27/35] More bug fixes for phantom stages. --- .../src/main/scala/org/apache/spark/ui/jobs/JobPage.scala | 8 ++++++-- .../test/scala/org/apache/spark/ui/UISeleniumSuite.scala | 6 ++++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index c02277120ada..03414dfadc9e 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 @@ -56,8 +56,12 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val completedStages = mutable.Buffer[StageInfo]() val failedStages = mutable.Buffer[StageInfo]() for (stage <- stages) { - if (!isComplete && stage.submissionTime.isEmpty) { - pendingStages += stage + if (stage.submissionTime.isEmpty) { + if (!isComplete) { + pendingStages += stage + } else { + // Do nothing so that we don't display pending stages for completed jobs + } } else if (stage.completionTime.isDefined) { if (stage.failureReason.isDefined) { failedStages += stage diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index f116e4e821a2..c0bb9345eee0 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -231,16 +231,18 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { // mentioned in its job start event but which were never actually executed: rdd.count() rdd.count() - // Check that the "pending stages" section is empty: eventually(timeout(10 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") find(id("pending")).get.text should be ("Pending Stages (0)") + find(id("active")).get.text should be ("Active Stages (0)") + find(id("completed")).get.text should be ("Completed Stages (1)") + find(id("failed")).get.text should be ("Failed Stages (0)") } } } - test("jobs with stages thatare skipped should show correct link descriptions on jobs page") { + test("jobs with stages that are skipped should show correct link descriptions on all jobs page") { withSpark(newSparkContext()) { sc => // Create an RDD that involves multiple stages: val rdd = From 1f45d44dfa45d2a8174837d36aa31ec31e4a1e80 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 20 Nov 2014 13:51:21 -0800 Subject: [PATCH 28/35] Incorporate a bunch of minor review feedback. --- .../spark/ui/jobs/JobProgressListener.scala | 21 ++++++++++++------- .../org/apache/spark/ui/jobs/UIData.scala | 7 +++++-- .../org/apache/spark/ui/UISeleniumSuite.scala | 2 +- 3 files changed, 20 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index a8941fdb8e93..31ed997ba4e8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -49,8 +49,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { type PoolName = String type ExecutorId = String - // Define all of our state: - // Jobs: val activeJobs = new HashMap[JobId, JobUIData] val completedJobs = ListBuffer[JobUIData]() @@ -151,9 +149,18 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) ) yield group val jobData: JobUIData = - new JobUIData(jobStart.jobId, Some(System.currentTimeMillis), None, jobStart.stageIds, - jobGroup, JobExecutionStatus.RUNNING) - // Compute (a potential underestimate of) the number of tasks that will be run by this job: + new JobUIData( + jobId = jobStart.jobId, + startTime = Some(System.currentTimeMillis), + endTime = None, + stageIds = jobStart.stageIds, + jobGroup = jobGroup, + status = JobExecutionStatus.RUNNING) + // Compute (a potential underestimate of) the number of tasks that will be run by this job. + // This may be an underestimate because the job start event references all of the result + // stages's transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. jobData.numTasks = { val allStages = jobStart.stageInfos val missingStages = allStages.filter(_.completionTime.isEmpty) @@ -195,8 +202,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // If this is a pending stage and no other job depends on it, then it won't be run. // To prevent memory leaks, remove this data since it won't be cleaned up as stages // finish / fail: - if (stageInfo.submissionTime.isEmpty && stageInfo.completionTime.isEmpty - && jobsUsingStage.isEmpty) { + val isPendingStage = stageInfo.submissionTime.isEmpty && stageInfo.completionTime.isEmpty + if (isPendingStage && jobsUsingStage.isEmpty) { stageIdToInfo.remove(stageId) stageIdToData.remove((stageId, stageInfo.attemptId)) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index f849792da455..79ec72e8f273 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -46,8 +46,11 @@ private[jobs] object UIData { var jobGroup: Option[String] = None, var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, /* Tasks */ - // `numTasks` is a potential underestimate of the true number of tasks that this job will run - // see https://github.com/apache/spark/pull/3009 for an extensive discussion of this + // `numTasks` is a potential underestimate of the true number of tasks that this job will run. + // This may be an underestimate because the job start event references all of the result + // stages's transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. var numTasks: Int = 0, var numActiveTasks: Int = 0, var numCompletedTasks: Int = 0, diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index c0bb9345eee0..ee0d7433a310 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -24,10 +24,10 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ -import org.apache.spark.api.java.StorageLevels import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.api.java.StorageLevels import org.apache.spark.shuffle.FetchFailedException /** From 61c265aac63d1621bf212245ddb25b703cf3a807 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 20 Nov 2014 14:27:51 -0800 Subject: [PATCH 29/35] =?UTF-8?q?Add=20=E2=80=9Cskipped=20stages=E2=80=9D?= =?UTF-8?q?=20table;=20only=20display=20non-empty=20tables.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../org/apache/spark/ui/jobs/JobPage.scala | 108 ++++++++++++------ .../spark/ui/jobs/JobProgressListener.scala | 13 +-- .../org/apache/spark/ui/UISeleniumSuite.scala | 20 +++- 3 files changed, 95 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 03414dfadc9e..08f2098e37a6 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 @@ -51,17 +51,14 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) } - val pendingStages = mutable.Buffer[StageInfo]() val activeStages = mutable.Buffer[StageInfo]() val completedStages = mutable.Buffer[StageInfo]() + // If the job is completed, then any pending stages are displayed as "skipped": + val pendingOrSkippedStages = mutable.Buffer[StageInfo]() val failedStages = mutable.Buffer[StageInfo]() for (stage <- stages) { if (stage.submissionTime.isEmpty) { - if (!isComplete) { - pendingStages += stage - } else { - // Do nothing so that we don't display pending stages for completed jobs - } + pendingOrSkippedStages += stage } else if (stage.completionTime.isDefined) { if (stage.failureReason.isDefined) { failedStages += stage @@ -77,8 +74,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = parent.killEnabled) - val pendingStagesTable = - new StageTableBase(pendingStages.sortBy(_.stageId).reverse, + val pendingOrSkippedStagesTable = + new StageTableBase(pendingOrSkippedStages.sortBy(_.stageId).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val completedStagesTable = @@ -89,6 +86,12 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler) + 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 =
        @@ -102,36 +105,73 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { Job Group: {jobData.jobGroup.get} - } else Seq.empty + } + } + { + if (shouldShowActiveStages) { +
      • + Active Stages: + {activeStages.size} +
      • + } + } + { + if (shouldShowPendingStages) { +
      • + + Pending Stages: + {pendingOrSkippedStages.size} +
      • + } + } + { + if (shouldShowCompletedStages) { +
      • + Completed Stages: + {completedStages.size} +
      • + } + } + { + if (shouldShowSkippedStages) { +
      • + Skipped Stages: + {pendingOrSkippedStages.size} +
      • + } + } + { + if (shouldShowFailedStages) { +
      • + Failed Stages: + {failedStages.size} +
      • + } } -
      • - Active Stages: - {activeStages.size} -
      • -
      • - Pending Stages: - {pendingStages.size} -
      • -
      • - Completed Stages: - {completedStages.size} -
      • -
      • - Failed Stages: - {failedStages.size} -
      - val content = summary ++ -

      Active Stages ({activeStages.size})

      ++ - activeStagesTable.toNodeSeq ++ -

      Pending Stages ({pendingStages.size})

      ++ - pendingStagesTable.toNodeSeq ++ -

      Completed Stages ({completedStages.size})

      ++ - completedStagesTable.toNodeSeq ++ -

      Failed Stages ({failedStages.size})

      ++ - failedStagesTable.toNodeSeq + var content = summary + 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) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 31ed997ba4e8..6e646e0210a7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -58,6 +58,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // Stages: val activeStages = new HashMap[StageId, StageInfo] val completedStages = ListBuffer[StageInfo]() + val skippedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData] val stageIdToInfo = new HashMap[StageId, StageInfo] @@ -106,6 +107,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { "completedJobs" -> completedJobs.size, "failedJobs" -> failedJobs.size, "completedStages" -> completedStages.size, + "skippedStages" -> skippedStages.size, "failedStages" -> failedStages.size ) } @@ -199,13 +201,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => jobsUsingStage.remove(jobEnd.jobId) stageIdToInfo.get(stageId).foreach { stageInfo => - // If this is a pending stage and no other job depends on it, then it won't be run. - // To prevent memory leaks, remove this data since it won't be cleaned up as stages - // finish / fail: - val isPendingStage = stageInfo.submissionTime.isEmpty && stageInfo.completionTime.isEmpty - if (isPendingStage && jobsUsingStage.isEmpty) { - stageIdToInfo.remove(stageId) - stageIdToData.remove((stageId, stageInfo.attemptId)) + if (stageInfo.submissionTime.isEmpty) { + // if this stage is pending, it won't complete, so mark it as "skipped": + skippedStages += stageInfo + trimStagesIfNecessary(skippedStages) } } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index ee0d7433a310..96370a55a8f3 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -210,6 +210,8 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { rdd.countAsync() eventually(timeout(10 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0") + find(id("active")).get.text should be ("Active Stages (1)") + find(id("pending")).get.text should be ("Pending Stages (2)") // Essentially, we want to check that none of the stage rows show // "No data available for this stage". Checking for the absence of that string is brittle // because someone could change the error message and cause this test to pass by accident. @@ -222,7 +224,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } - test("stages that aren't run do not show up in 'pending stages' after a job finishes") { + test("stages that aren't run appear as 'skipped stages' after a job finishes") { withSpark(newSparkContext()) { sc => // Create an RDD that involves multiple stages: val rdd = @@ -233,15 +235,23 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { rdd.count() eventually(timeout(10 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") - find(id("pending")).get.text should be ("Pending Stages (0)") - find(id("active")).get.text should be ("Active Stages (0)") + find(id("pending")) should be (None) + find(id("active")) should be (None) + find(id("failed")) should be (None) find(id("completed")).get.text should be ("Completed Stages (1)") - find(id("failed")).get.text should be ("Failed Stages (0)") + find(id("skipped")).get.text should be ("Skipped Stages (2)") + // Essentially, we want to check that none of the stage rows show + // "No data available for this stage". Checking for the absence of that string is brittle + // because someone could change the error message and cause this test to pass by accident. + // Instead, it's safer to check that each row contains a link to a stage details page. + findAll(cssSelector("tbody tr")).foreach { row => + val link = row.underlying.findElement(By.xpath(".//a")) + link.getAttribute("href") should include("stage") + } } } } - test("jobs with stages that are skipped should show correct link descriptions on all jobs page") { withSpark(newSparkContext()) { sc => // Create an RDD that involves multiple stages: From 2bbf41a9845c87f0facfb737f26a5b52ba6633ef Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 20 Nov 2014 15:06:00 -0800 Subject: [PATCH 30/35] Update job progress bar to reflect skipped tasks/stages. --- .../scala/org/apache/spark/ui/UIUtils.scala | 11 +++++-- .../apache/spark/ui/jobs/AllJobsPage.scala | 14 ++++---- .../spark/ui/jobs/JobProgressListener.scala | 2 ++ .../org/apache/spark/ui/jobs/StageTable.scala | 5 +-- .../org/apache/spark/ui/jobs/UIData.scala | 3 ++ .../org/apache/spark/ui/UISeleniumSuite.scala | 32 ++++++++++++++++++- 6 files changed, 56 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index eb0cb759d6a2..0c418beaf758 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -289,13 +289,20 @@ private[spark] object UIUtils extends Logging { } - def makeProgressBar(started: Int, completed: Int, failed: Int, total: Int): Seq[Node] = { + def makeProgressBar( + started: Int, + completed: Int, + failed: Int, + skipped:Int, + total: Int): Seq[Node] = { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100)
      - {completed}/{total} { if (failed > 0) s"($failed failed)" else "" } + {completed}/{total} + { if (failed > 0) s"($failed failed)" } + { if (skipped > 0) s"($skipped skipped)" }
      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 286736ff1b70..ea2d187a0e8e 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 @@ -21,10 +21,10 @@ import scala.xml.{Node, NodeSeq} import javax.servlet.http.HttpServletRequest +import org.apache.spark.JobExecutionStatus import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData.JobUIData - /** Page showing list of all ongoing and recently finished jobs */ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { private val startTime: Option[Long] = parent.sc.map(_.startTime) @@ -47,6 +47,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val lastStageData = lastStageInfo.flatMap { s => listener.stageIdToData.get((s.stageId, s.attemptId)) } + val isComplete = job.status == JobExecutionStatus.SUCCEEDED val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") val duration: Option[Long] = { @@ -59,7 +60,6 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val formattedSubmissionTime = job.startTime.map(UIUtils.formatDate).getOrElse("Unknown") val detailUrl = "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) - {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} @@ -73,12 +73,14 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {formattedDuration} - {job.completedStageIndices.size}/{job.stageIds.size} - {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)" else ""} + {job.completedStageIndices.size}/{job.stageIds.size - job.numSkippedStages} + {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} + {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} - {UIUtils.makeProgressBar(job.numActiveTasks, job.numCompletedTasks, - job.numFailedTasks, job.numTasks)} + {UIUtils.makeProgressBar(started = job.numActiveTasks, completed = job.numCompletedTasks, + failed = job.numFailedTasks, skipped = job.numSkippedTasks, + total = job.numTasks - job.numSkippedTasks)} } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 6e646e0210a7..72935beb3a34 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -205,6 +205,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // if this stage is pending, it won't complete, so mark it as "skipped": skippedStages += stageInfo trimStagesIfNecessary(skippedStages) + jobData.numSkippedStages += 1 + jobData.numSkippedTasks += stageInfo.numTasks } } } 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 f2f97a5015bb..e7d6244dcd67 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 @@ -165,8 +165,9 @@ private[ui] class StageTableBase( {formattedDuration} - {UIUtils.makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size, - stageData.numFailedTasks, s.numTasks)} + {UIUtils.makeProgressBar(started = stageData.numActiveTasks, + completed = stageData.completedIndices.size, failed = stageData.numFailedTasks, + skipped = 0, total = s.numTasks)} {inputReadWithUnit} {outputWriteWithUnit} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 79ec72e8f273..48fd7caa1a1e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -54,10 +54,13 @@ private[jobs] object UIData { var numTasks: Int = 0, var numActiveTasks: Int = 0, var numCompletedTasks: Int = 0, + var numSkippedTasks: Int = 0, var numFailedTasks: Int = 0, /* Stages */ var numActiveStages: Int = 0, + // This needs to be a set instead of a simple count to prevent double-counting of rerun stages: var completedStageIndices: OpenHashSet[Int] = new OpenHashSet[Int](), + var numSkippedStages: Int = 0, var numFailedStages: Int = 0 ) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 96370a55a8f3..d2857b8b5566 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.ui +import scala.collection.JavaConversions._ + import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest._ @@ -224,6 +226,34 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } + test("job progress bars / cells reflect skipped stages / tasks") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = sc.parallelize(1 to 8, 8) + .map(x => x).groupBy((x: Int) => x, numPartitions = 8) + .flatMap(x => x._2).groupBy((x: Int) => x, numPartitions = 8) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + // The completed jobs table should have two rows. The first row will be the most recent job: + val firstRow = find(cssSelector("tbody tr")).get.underlying + val firstRowColumns = firstRow.findElements(By.tagName("td")) + firstRowColumns(0).getText should be ("1") + firstRowColumns(4).getText should be ("1/1 (2 skipped)") + firstRowColumns(5).getText should be ("8/8 (16 skipped)") + // The second row is the first run of the job, where nothing was skipped: + val secondRow = findAll(cssSelector("tbody tr")).toSeq(1).underlying + val secondRowColumns = secondRow.findElements(By.tagName("td")) + secondRowColumns(0).getText should be ("0") + secondRowColumns(4).getText should be ("3/3") + secondRowColumns(5).getText should be ("24/24") + } + } + } + test("stages that aren't run appear as 'skipped stages' after a job finishes") { withSpark(newSparkContext()) { sc => // Create an RDD that involves multiple stages: @@ -246,7 +276,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { // Instead, it's safer to check that each row contains a link to a stage details page. findAll(cssSelector("tbody tr")).foreach { row => val link = row.underlying.findElement(By.xpath(".//a")) - link.getAttribute("href") should include("stage") + link.getAttribute("href") should include ("stage") } } } From 6f17f3f61102f5685d20cf42f79a049a5bbaad06 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 20 Nov 2014 16:49:32 -0800 Subject: [PATCH 31/35] Only store StageInfos in SparkListenerJobStart event. --- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../spark/scheduler/SparkListener.scala | 7 +- .../org/apache/spark/util/JsonProtocol.scala | 18 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 190 ++++++++++++++++-- 5 files changed, 193 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 94d11178ade4..b1222af662e9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -751,7 +751,7 @@ class DAGScheduler( localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Seq.empty, Seq.empty, properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Seq.empty, properties)) runLocally(job) } else { jobIdToActiveJob(jobId) = job @@ -759,7 +759,7 @@ class DAGScheduler( finalStage.resultOfJob = Some(job) val stageIds = jobIdToStageIds(jobId).toArray val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) - listenerBus.post(SparkListenerJobStart(job.jobId, stageInfos, stageIds, properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, stageInfos, properties)) submitStage(finalStage) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 6b46a94d9fc9..b62b0c131269 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -59,9 +59,12 @@ case class SparkListenerTaskEnd( case class SparkListenerJobStart( jobId: Int, stageInfos: Seq[StageInfo], - stageIds: Seq[Int], // Note: this is here for backwards-compatibility properties: Properties = null) - extends SparkListenerEvent + extends SparkListenerEvent { + // Note: this is here for backwards-compatibility with older versions of this event which + // only stored stageIds and not StageInfos: + val stageIds: Seq[Int] = stageInfos.map(_.stageId) +} @DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 16dec5f51a86..29428ca6f62d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -454,11 +454,21 @@ private[spark] object JsonProtocol { def jobStartFromJson(json: JValue): SparkListenerJobStart = { val jobId = (json \ "Job ID").extract[Int] - val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") - val stageInfos = Utils.jsonOption(json \ "Stage Infos") - .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse(Seq.empty) - SparkListenerJobStart(jobId, stageInfos, stageIds, properties) + val stageInfos = { + // Prior to 1.2.0, we serialized stageIds but not stageInfos; in 1.2.0, we do the opposite. + // This block of code handles backwards compatibility: + val stageIds: Option[Seq[Int]] = + Utils.jsonOption(json \ "Stage IDs").map(_.extract[List[JValue]].map(_.extract[Int])) + if (stageIds.isDefined) { + stageIds.get.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + } else { + Utils.jsonOption(json \ "Stage Infos") + .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse(Seq.empty) + } + } + + SparkListenerJobStart(jobId, stageInfos, properties) } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 53e7b9b4e613..12af60caf7d5 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -46,7 +46,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val stageInfos = stageIds.map { stageId => new StageInfo(stageId, 0, stageId.toString, 0, null, "") } - SparkListenerJobStart(jobId, stageInfos, stageIds) + SparkListenerJobStart(jobId, stageInfos) } private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index d2510ec8092a..11f428d07f23 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -49,9 +49,8 @@ class JsonProtocolSuite extends FunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)) val jobStart = { val stageIds = Seq[Int](1, 2, 3, 4) - val stageInfos = stageIds.map(x => - makeStageInfo(x * 100, x * 200, x * 300, x * 400L, x * 500L)) - SparkListenerJobStart(10, stageInfos, stageIds, properties) + val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) + SparkListenerJobStart(10, stageInfos, properties) } val jobEnd = SparkListenerJobEnd(20, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( @@ -231,15 +230,12 @@ class JsonProtocolSuite extends FunSuite { test("SparkListenerJobStart backward compatibility") { // Prior to Spark 1.2.0, SparkListenerJobStart did not have a "Stage Infos" property. - val jobStart = { - val stageIds = Seq[Int](1, 2, 3, 4) - val stageInfos = stageIds.map(x => - makeStageInfo(x * 100, x * 200, x * 300, x * 400L, x * 500L)) - SparkListenerJobStart(10, stageInfos, stageIds, properties) - } + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + val jobStart = SparkListenerJobStart(10, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) - val expectedJobStart = SparkListenerJobStart(10, Seq.empty, Seq[Int](1, 2, 3, 4), properties) - assert(expectedJobStart === JsonProtocol.jobStartFromJson(oldEvent)) + val expectedJobStart = SparkListenerJobStart(10, stageInfos, properties) + assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent)) } /** -------------------------- * @@ -1071,12 +1067,27 @@ class JsonProtocolSuite extends FunSuite { | "Job ID": 10, | "Stage Infos": [ | { - | "Stage ID": 100, + | "Stage ID": 1, | "Stage Attempt ID": 0, | "Stage Name": "greetings", | "Number of Tasks": 200, | "RDD Info": [ - | + | { + | "RDD ID": 1, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 200, + | "Number of Cached Partitions": 300, + | "Memory Size": 400, + | "Tachyon Size": 0, + | "Disk Size": 500 + | } | ], | "Details": "details", | "Accumulables": [ @@ -1095,12 +1106,43 @@ class JsonProtocolSuite extends FunSuite { | ] | }, | { - | "Stage ID": 200, + | "Stage ID": 2, | "Stage Attempt ID": 0, | "Stage Name": "greetings", | "Number of Tasks": 400, | "RDD Info": [ - | + | { + | "RDD ID": 2, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 400, + | "Number of Cached Partitions": 600, + | "Memory Size": 800, + | "Tachyon Size": 0, + | "Disk Size": 1000 + | }, + | { + | "RDD ID": 3, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 401, + | "Number of Cached Partitions": 601, + | "Memory Size": 801, + | "Tachyon Size": 0, + | "Disk Size": 1001 + | } | ], | "Details": "details", | "Accumulables": [ @@ -1119,12 +1161,59 @@ class JsonProtocolSuite extends FunSuite { | ] | }, | { - | "Stage ID": 300, + | "Stage ID": 3, | "Stage Attempt ID": 0, | "Stage Name": "greetings", | "Number of Tasks": 600, | "RDD Info": [ - | + | { + | "RDD ID": 3, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 600, + | "Number of Cached Partitions": 900, + | "Memory Size": 1200, + | "Tachyon Size": 0, + | "Disk Size": 1500 + | }, + | { + | "RDD ID": 4, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 601, + | "Number of Cached Partitions": 901, + | "Memory Size": 1201, + | "Tachyon Size": 0, + | "Disk Size": 1501 + | }, + | { + | "RDD ID": 5, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 602, + | "Number of Cached Partitions": 902, + | "Memory Size": 1202, + | "Tachyon Size": 0, + | "Disk Size": 1502 + | } | ], | "Details": "details", | "Accumulables": [ @@ -1143,12 +1232,75 @@ class JsonProtocolSuite extends FunSuite { | ] | }, | { - | "Stage ID": 400, + | "Stage ID": 4, | "Stage Attempt ID": 0, | "Stage Name": "greetings", | "Number of Tasks": 800, | "RDD Info": [ - | + | { + | "RDD ID": 4, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 800, + | "Number of Cached Partitions": 1200, + | "Memory Size": 1600, + | "Tachyon Size": 0, + | "Disk Size": 2000 + | }, + | { + | "RDD ID": 5, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 801, + | "Number of Cached Partitions": 1201, + | "Memory Size": 1601, + | "Tachyon Size": 0, + | "Disk Size": 2001 + | }, + | { + | "RDD ID": 6, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 802, + | "Number of Cached Partitions": 1202, + | "Memory Size": 1602, + | "Tachyon Size": 0, + | "Disk Size": 2002 + | }, + | { + | "RDD ID": 7, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 803, + | "Number of Cached Partitions": 1203, + | "Memory Size": 1603, + | "Tachyon Size": 0, + | "Disk Size": 2003 + | } | ], | "Details": "details", | "Accumulables": [ From ff804cd3699218fccdd191743af7ff855a0235f1 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 20 Nov 2014 22:13:17 -0800 Subject: [PATCH 32/35] Don't write "Stage Ids" field in JobStartEvent JSON. --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 29428ca6f62d..5123df0bcfe0 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -122,7 +122,6 @@ private[spark] object JsonProtocol { ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ - ("Stage IDs" -> jobStart.stageIds) ~ ("Properties" -> properties) } From b89c2587efb52ab3f5d8fc1a60fbd5a4f9c07510 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 20 Nov 2014 23:44:38 -0800 Subject: [PATCH 33/35] More JSON protocol backwards-compatibility fixes. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The root problem was that I removed a field from the JSON and recomputed it from another field. In order for the backwards-compatibility test to work, I needed to manually re-add the removed field in order to construct JSON that’s in the right (old) format. --- .../scala/org/apache/spark/util/JsonProtocol.scala | 7 ++++--- .../org/apache/spark/util/JsonProtocolSuite.scala | 11 ++++------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 5123df0bcfe0..9d1475cf1b47 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -121,7 +121,8 @@ private[spark] object JsonProtocol { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ - ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ + // ("Stage IDs" -> jobStart.stageIds) ~ // Removed in 1.2.0 + ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in 1.2.0 ("Properties" -> properties) } @@ -459,9 +460,9 @@ private[spark] object JsonProtocol { // This block of code handles backwards compatibility: val stageIds: Option[Seq[Int]] = Utils.jsonOption(json \ "Stage IDs").map(_.extract[List[JValue]].map(_.extract[Int])) - if (stageIds.isDefined) { + if (stageIds.isDefined) { // Reading JSON written prior to 1.2.0 stageIds.get.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) - } else { + } else { // Reading JSON written after 1.2.0 Utils.jsonOption(json \ "Stage Infos") .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse(Seq.empty) } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 11f428d07f23..3e5e8b1e8454 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -21,6 +21,8 @@ import java.util.Properties import scala.collection.Map +import org.json4s.JsonAST.JObject +import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.scalatest.FunSuite @@ -234,6 +236,7 @@ class JsonProtocolSuite extends FunSuite { val stageInfos = stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) val jobStart = SparkListenerJobStart(10, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) + .asInstanceOf[JObject] ~ ("Stage IDs" -> stageIds) val expectedJobStart = SparkListenerJobStart(10, stageInfos, properties) assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent)) } @@ -320,7 +323,7 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => assert(e1.jobId === e2.jobId) assert(e1.properties === e2.properties) - assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 === i2)) + assert(e1.stageIds === e2.stageIds) case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) => assert(e1.jobId === e2.jobId) assertEquals(e1.jobResult, e2.jobResult) @@ -1319,12 +1322,6 @@ class JsonProtocolSuite extends FunSuite { | ] | } | ], - | "Stage IDs": [ - | 1, - | 2, - | 3, - | 4 - | ], | "Properties": { | "France": "Paris", | "Germany": "Berlin", From f00c851c2be2f104eb99560e2228b2ffe45fce95 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 21 Nov 2014 15:45:11 -0800 Subject: [PATCH 34/35] Fix JsonProtocol compatibility --- .../org/apache/spark/util/JsonProtocol.scala | 36 +++++++++++-------- .../apache/spark/util/JsonProtocolSuite.scala | 19 ++++++---- 2 files changed, 35 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 9d1475cf1b47..7b5db1ed7626 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -31,6 +31,21 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ +/** + * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- + * and forwards-compatibility guarantees: any version of Spark should be able to read JSON output + * written by any other version, including newer versions. + * + * JsonProtocolSuite contains backwards-compatibility tests which check that the current version of + * JsonProtocol is able to read output written by earlier versions. We do not currently have tests + * for reading newer JSON output with older Spark versions. + * + * To ensure that we provide these guarantees, follow these rules when modifying these methods: + * + * - Never delete any JSON fields. + * - Any new JSON fields should be optional; use `Utils.jsonOption` when reading these fields + * in `*FromJson` methods. + */ private[spark] object JsonProtocol { // TODO: Remove this file and put JSON serialization into each individual class. @@ -121,8 +136,8 @@ private[spark] object JsonProtocol { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ - // ("Stage IDs" -> jobStart.stageIds) ~ // Removed in 1.2.0 - ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in 1.2.0 + ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in Spark 1.2.0 + ("Stage IDs" -> jobStart.stageIds) ~ ("Properties" -> properties) } @@ -454,20 +469,13 @@ private[spark] object JsonProtocol { def jobStartFromJson(json: JValue): SparkListenerJobStart = { val jobId = (json \ "Job ID").extract[Int] + val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") - val stageInfos = { - // Prior to 1.2.0, we serialized stageIds but not stageInfos; in 1.2.0, we do the opposite. - // This block of code handles backwards compatibility: - val stageIds: Option[Seq[Int]] = - Utils.jsonOption(json \ "Stage IDs").map(_.extract[List[JValue]].map(_.extract[Int])) - if (stageIds.isDefined) { // Reading JSON written prior to 1.2.0 - stageIds.get.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) - } else { // Reading JSON written after 1.2.0 - Utils.jsonOption(json \ "Stage Infos") - .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse(Seq.empty) + // The "Stage Infos" field was added in Spark 1.2.0 + val stageInfos = Utils.jsonOption(json \ "Stage Infos") + .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) } - } - SparkListenerJobStart(jobId, stageInfos, properties) } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 3e5e8b1e8454..0bc949267586 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -21,8 +21,6 @@ import java.util.Properties import scala.collection.Map -import org.json4s.JsonAST.JObject -import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.scalatest.FunSuite @@ -51,7 +49,8 @@ class JsonProtocolSuite extends FunSuite { makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)) val jobStart = { val stageIds = Seq[Int](1, 2, 3, 4) - val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) + val stageInfos = stageIds.map(x => + makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) SparkListenerJobStart(10, stageInfos, properties) } val jobEnd = SparkListenerJobEnd(20, JobSucceeded) @@ -233,11 +232,13 @@ class JsonProtocolSuite extends FunSuite { test("SparkListenerJobStart backward compatibility") { // Prior to Spark 1.2.0, SparkListenerJobStart did not have a "Stage Infos" property. val stageIds = Seq[Int](1, 2, 3, 4) - val stageInfos = stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400, x * 500)) + val dummyStageInfos = + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) val jobStart = SparkListenerJobStart(10, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) - .asInstanceOf[JObject] ~ ("Stage IDs" -> stageIds) - val expectedJobStart = SparkListenerJobStart(10, stageInfos, properties) + val expectedJobStart = + SparkListenerJobStart(10, dummyStageInfos, properties) assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent)) } @@ -1322,6 +1323,12 @@ class JsonProtocolSuite extends FunSuite { | ] | } | ], + | "Stage IDs": [ + | 1, + | 2, + | 3, + | 4 + | ], | "Properties": { | "France": "Paris", | "Germany": "Berlin", From eb05e9044c700c2d2cabdf2fd1cdc89554c05bb4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 23 Nov 2014 22:59:17 -0800 Subject: [PATCH 35/35] Disable kill button in completed stages tables. --- .../main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala | 3 +-- core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) 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 908c96dfae49..b0f8ca2ab0d3 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 @@ -45,8 +45,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { killEnabled = parent.killEnabled) val completedStagesTable = new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, isFairScheduler = parent.isFairScheduler, - killEnabled = parent.killEnabled) + parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val failedStagesTable = new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler) 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 08f2098e37a6..77d36209c604 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 @@ -80,8 +80,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { killEnabled = false) val completedStagesTable = new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, - parent.listener, isFairScheduler = parent.isFairScheduler, - killEnabled = parent.killEnabled) + parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val failedStagesTable = new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler)