[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63844350
  
I looked through this and took a spin locally, LGTM.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63851287
  
Hey @JoshRosen I'll take a look at this right now. Is it still WIP by the 
way?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20664971
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -144,11 +146,30 @@ class JobProgressListener(conf: SparkConf) extends 
SparkListener with Logging {
   }
 
   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)
+// Compute (a potential underestimate of) the number of tasks that 
will be run by this job:
--- End diff --

I think it would be good to explain briefly why it can potentially be an 
underestimate


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665049
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -166,6 +188,21 @@ class JobProgressListener(conf: SparkConf) extends 
SparkListener with Logging {
 trimJobsIfNecessary(failedJobs)
 jobData.status = JobExecutionStatus.FAILED
 }
+for (stageId - jobData.stageIds) {
+  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) {
--- End diff --

this looks funky, can you do it like this
```
if (stageInfo.submissionTime.isEmpty 
stageInfo.completionTime.isEmpty 
jobsUsingStage.isEmpty) {
  ...
}
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665136
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala ---
@@ -40,9 +40,22 @@ 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
+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
--- End diff --

as in above, is it possible to provide a 1-line quick summary of why that 
is the case, and if the reader wants to dig deeper then they can follow the 
link?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665208
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -214,6 +264,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.numActiveStages += 1
+}
--- End diff --

So what's the behavior now for resubmitted stages? Will they result in # 
completed stages  # total stages in the UI (and similarly for # tasks)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665341
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -214,6 +264,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.numActiveStages += 1
+}
--- End diff --

Ok I realized what this is (the apparent-hang behavior we discussed) -- but 
it would be great to add a comment somewhere describing this (maybe in 
JobUIData, explaining why completedStages is a set?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665377
  
--- Diff: core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala ---
@@ -17,16 +17,18 @@
 
 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.{By, WebDriver}
 import org.openqa.selenium.htmlunit.HtmlUnitDriver
 import org.scalatest._
 import org.scalatest.concurrent.Eventually._
 import org.scalatest.selenium.WebBrowser
 import org.scalatest.time.SpanSugar._
 
+import org.apache.spark.api.java.StorageLevels
--- End diff --

nit: this should be below the next three


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665632
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -144,11 +146,30 @@ class JobProgressListener(conf: SparkConf) extends 
SparkListener with Logging {
   }
 
   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,
--- End diff --

Can you name each of these parameters? As is, I find it hard to read


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665705
  
--- Diff: 
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
--- End diff --

Why do you need this for backwards compatibility? Can the JsonProtocol do 
something smarter where it just fills in the StageInfos with 0 except for the 
stageIds?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665717
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala ---
@@ -0,0 +1,149 @@
+/*
+ * 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 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 jobs */
+private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage() {
+  private val startTime: Option[Long] = parent.sc.map(_.startTime)
+  private val listener = parent.listener
+
+  private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = {
+val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined)
+
+val columns: Seq[Node] = {
+  th{if (someJobHasJobGroup) Job Id (Job Group) else Job Id}/th
+  thDescription/th
+  thSubmitted/th
+  thDuration/th
+  th class=sorttable_nosortStages: Succeeded/Total/th
+  th class=sorttable_nosortTasks (for all stages): 
Succeeded/Total/th
+}
+
+def makeRow(job: JobUIData): Seq[Node] = {
+  val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max)
+  val lastStageData = lastStageInfo.flatMap { s =
+listener.stageIdToData.get((s.stageId, s.attemptId))
+  }
+  val lastStageName = lastStageInfo.map(_.name).getOrElse((Unknown 
Stage Name))
+  val lastStageDescription = 
lastStageData.flatMap(_.description).getOrElse()
+  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)
--- End diff --

I realize we use `Unknown` in a few places. Can you declare a
```
val UNKNOWN: String = Unknown
```
in `UIUtils`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20665799
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala ---
@@ -0,0 +1,149 @@
+/*
+ * 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 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 jobs */
+private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage() {
+  private val startTime: Option[Long] = parent.sc.map(_.startTime)
+  private val listener = parent.listener
+
+  private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = {
+val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined)
+
+val columns: Seq[Node] = {
+  th{if (someJobHasJobGroup) Job Id (Job Group) else Job Id}/th
+  thDescription/th
+  thSubmitted/th
+  thDuration/th
+  th class=sorttable_nosortStages: Succeeded/Total/th
+  th class=sorttable_nosortTasks (for all stages): 
Succeeded/Total/th
+}
+
+def makeRow(job: JobUIData): Seq[Node] = {
+  val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max)
+  val lastStageData = lastStageInfo.flatMap { s =
+listener.stageIdToData.get((s.stageId, s.attemptId))
+  }
+  val lastStageName = lastStageInfo.map(_.name).getOrElse((Unknown 
Stage Name))
+  val lastStageDescription = 
lastStageData.flatMap(_.description).getOrElse()
+  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)
+
+  tr
+td sorttable_customkey={job.jobId.toString}
+  {job.jobId} {job.jobGroup.map(id = s($id)).getOrElse()}
+/td
+td
+  divem{lastStageDescription}/em/div
+  a href={detailUrl}{lastStageName}/a
+/td
+td sorttable_customkey={job.startTime.getOrElse(-1).toString}
+  {formattedSubmissionTime}
+/td
+td 
sorttable_customkey={duration.getOrElse(-1).toString}{formattedDuration}/td
+td class=stage-progress-cell
+  {job.completedStageIndices.size}/{job.stageIds.size}
+  {if (job.numFailedStages  0) s(${job.numFailedStages} failed) 
else }
+/td
+td class=progress-cell
+  {UIUtils.makeProgressBar(job.numActiveTasks, 
job.numCompletedTasks,
+  job.numFailedTasks, job.numTasks)}
+/td
+  /tr
+}
+
+table class=table table-bordered table-striped table-condensed 
sortable
+  thead{columns}/thead
+  tbody
+{jobs.map(makeRow)}
+  /tbody
+/table
+  }
+
+  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(_.startTime.getOrElse(-1L)).reverse)
+  val completedJobsTable =
+jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse)
+  val failedJobsTable =
+jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse)
+
+  val summary: NodeSeq =
  

[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread kayousterhout
Github user kayousterhout commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63854023
  
All superficial comments on the code, but I tried it out and still got this 
somewhat icky situation where a completed stage has fewer succeeded tasks than 
the total:


![image](https://cloud.githubusercontent.com/assets/1108612/5130344/a80bdca2-709e-11e4-905b-9e5baebec7c9.png)

Is this still the expected behavior? (this happened from running val rdd = 
sc.parallelize(1 to 10, 2).map((_, 1)).reduceByKey(_+_) and then counting the 
elements twice)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20666049
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala ---
@@ -195,9 +180,10 @@ private[ui] class StageTableBase(
 
 private[ui] class FailedStageTable(
 stages: Seq[StageInfo],
-parent: JobProgressTab,
-killEnabled: Boolean = false)
-  extends StageTableBase(stages, parent, killEnabled) {
+basePath: String,
+listener: JobProgressListener,
+isFairScheduler: Boolean)
+  extends StageTableBase(stages, basePath, listener, isFairScheduler, 
killEnabled = false) {
--- End diff --

Not your change, but weird how `killEnabled` is an attribute of the 
`StageTableBase`. The right thing to do is to have a `KillableStageTable` or 
something. We can fix this later (no action needed on your part)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread kayousterhout
Github user kayousterhout commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63855330
  
Also, I thought more about having this in 1.2, and I'm -0.5 on putting this 
in 1.2.  Given all of the subtleties you ended up running into with this, Josh, 
I don't think it's a good idea to put it in the release at the last minute 
without giving folks plenty of time to test it out.  Based on my (admittedly 
limited!) understanding of most Spark users, the UI page is one of the first 
thing a user will look at, and I don't think pushing a major change to what the 
user first sees when he or she interacts with Spark this late in the release 
cycle is a good idea.  If I were a Spark user and had successfully tried the 
preview release Patrick had posted, and then later found that the final 1.2 
release changed the landing page for the UI in a buggy / unintuitive / insert 
unexpected bug here, I think I'd be annoyed and question the Spark QA process.

I'm certainly willing to be overruled if others are less concerned about 
the points I mentioned.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread pwendell
Github user pwendell commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20669493
  
--- Diff: 
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
--- End diff --

The issue if someone with an older version of Spark reads a log message 
from a newer version. We can't go back and modify how older versions parse the 
fields, so we need to include fields expected by older versions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread pwendell
Github user pwendell commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20669574
  
--- Diff: 
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
--- End diff --

The problem I described could be fixed by just modifying the serializer and 
not the message here. However, we'd also like to provide binary compatiblity 
for people who wrote custom listeners and expect this field to be there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63862396
  
Hi Kay,

The behavior you noticed is intentional. If a job completes but doesn't run 
all of it's stages, it ends up showing as finished with a partially completed 
progress bar.

IMO - this should go in the release because it's one of the most requested 
features for Spark (giving better insights into runtime performance of a job). 
Because of the way the UI/Listners are structured it's very low risk and cannot 
interfere with other lower level functionality. Also, the existing Stage page 
is left unmodified, so this is purely additive.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20673960
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -144,11 +146,30 @@ class JobProgressListener(conf: SparkConf) extends 
SparkListener with Logging {
   }
 
   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)
+// Compute (a potential underestimate of) the number of tasks that 
will be run by this job:
--- End diff --

How's this for an explanation?

```
// 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.
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20674515
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala ---
@@ -0,0 +1,149 @@
+/*
+ * 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 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 jobs */
+private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage() {
+  private val startTime: Option[Long] = parent.sc.map(_.startTime)
+  private val listener = parent.listener
+
+  private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = {
+val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined)
+
+val columns: Seq[Node] = {
+  th{if (someJobHasJobGroup) Job Id (Job Group) else Job Id}/th
+  thDescription/th
+  thSubmitted/th
+  thDuration/th
+  th class=sorttable_nosortStages: Succeeded/Total/th
+  th class=sorttable_nosortTasks (for all stages): 
Succeeded/Total/th
+}
+
+def makeRow(job: JobUIData): Seq[Node] = {
+  val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max)
+  val lastStageData = lastStageInfo.flatMap { s =
+listener.stageIdToData.get((s.stageId, s.attemptId))
+  }
+  val lastStageName = lastStageInfo.map(_.name).getOrElse((Unknown 
Stage Name))
+  val lastStageDescription = 
lastStageData.flatMap(_.description).getOrElse()
+  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)
+
+  tr
+td sorttable_customkey={job.jobId.toString}
+  {job.jobId} {job.jobGroup.map(id = s($id)).getOrElse()}
+/td
+td
+  divem{lastStageDescription}/em/div
+  a href={detailUrl}{lastStageName}/a
+/td
+td sorttable_customkey={job.startTime.getOrElse(-1).toString}
+  {formattedSubmissionTime}
+/td
+td 
sorttable_customkey={duration.getOrElse(-1).toString}{formattedDuration}/td
+td class=stage-progress-cell
+  {job.completedStageIndices.size}/{job.stageIds.size}
+  {if (job.numFailedStages  0) s(${job.numFailedStages} failed) 
else }
+/td
+td class=progress-cell
+  {UIUtils.makeProgressBar(job.numActiveTasks, 
job.numCompletedTasks,
+  job.numFailedTasks, job.numTasks)}
+/td
+  /tr
+}
+
+table class=table table-bordered table-striped table-condensed 
sortable
+  thead{columns}/thead
+  tbody
+{jobs.map(makeRow)}
+  /tbody
+/table
+  }
+
+  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(_.startTime.getOrElse(-1L)).reverse)
+  val completedJobsTable =
+jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse)
+  val failedJobsTable =
+jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse)
+
+  val summary: NodeSeq =
   

[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63873718
  
@kayousterhout 

 Is this still the expected behavior? (this happened from running val rdd 
= sc.parallelize(1 to 10, 2).map((, 1)).reduceByKey(+_) and then counting the 
elements twice)

Yes, this is expected (I should probably write a Selenium test that 
explicitly defines this behavior in order to detect if it inadvertently 
changes).  Given that we have an overestimate of which stages will be run when 
the job starts, how would you change this?  One approach would be to just prune 
the stages that weren't run and advance the progress bar to 100% (e.g. it would 
show 1/1 stages and 2/2/ tasks for your example).  Another approach would be to 
enrich the listener API so that the UI can determine which stages are likely to 
be skipped and display a progress bar that's a potential underestimate.  If we 
do this, though, I think we'd want to have the progress bar update itself to 
show more remaining tasks once it learns that more stages need to be run.  This 
is going to require a _lot_ more testing to make sure that it doesn't run into 
any corner-cases.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20675128
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala ---
@@ -195,9 +180,10 @@ private[ui] class StageTableBase(
 
 private[ui] class FailedStageTable(
 stages: Seq[StageInfo],
-parent: JobProgressTab,
-killEnabled: Boolean = false)
-  extends StageTableBase(stages, parent, killEnabled) {
+basePath: String,
+listener: JobProgressListener,
+isFairScheduler: Boolean)
+  extends StageTableBase(stages, basePath, listener, isFairScheduler, 
killEnabled = false) {
--- End diff --

I'd be happy to chat offline about this, but I'm concerned that a 
`KillableStagesTable` might be very difficult to implement in our current UI 
framework because we don't have a great way to subclass / override portions of 
the HTML generation; there's not really any modularization / interfaces / etc. 
in any of our markup generation.

I played around with some ideas to improve this over at #2852 but 
eventually abandoned that approach because I ran into corner-cases that my 
abstraction couldn't cleanly handle.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20675254
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -144,11 +146,30 @@ class JobProgressListener(conf: SparkConf) extends 
SparkListener with Logging {
   }
 
   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,
--- End diff --

Good idea; it's more verbose but should help to avoid errors if we modify 
this later.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20677746
  
--- Diff: core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala ---
@@ -0,0 +1,149 @@
+/*
+ * 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 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 jobs */
+private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage() {
+  private val startTime: Option[Long] = parent.sc.map(_.startTime)
+  private val listener = parent.listener
+
+  private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = {
+val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined)
+
+val columns: Seq[Node] = {
+  th{if (someJobHasJobGroup) Job Id (Job Group) else Job Id}/th
+  thDescription/th
+  thSubmitted/th
+  thDuration/th
+  th class=sorttable_nosortStages: Succeeded/Total/th
+  th class=sorttable_nosortTasks (for all stages): 
Succeeded/Total/th
+}
+
+def makeRow(job: JobUIData): Seq[Node] = {
+  val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max)
+  val lastStageData = lastStageInfo.flatMap { s =
+listener.stageIdToData.get((s.stageId, s.attemptId))
+  }
+  val lastStageName = lastStageInfo.map(_.name).getOrElse((Unknown 
Stage Name))
+  val lastStageDescription = 
lastStageData.flatMap(_.description).getOrElse()
+  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)
+
+  tr
+td sorttable_customkey={job.jobId.toString}
+  {job.jobId} {job.jobGroup.map(id = s($id)).getOrElse()}
+/td
+td
+  divem{lastStageDescription}/em/div
+  a href={detailUrl}{lastStageName}/a
+/td
+td sorttable_customkey={job.startTime.getOrElse(-1).toString}
+  {formattedSubmissionTime}
+/td
+td 
sorttable_customkey={duration.getOrElse(-1).toString}{formattedDuration}/td
+td class=stage-progress-cell
+  {job.completedStageIndices.size}/{job.stageIds.size}
+  {if (job.numFailedStages  0) s(${job.numFailedStages} failed) 
else }
+/td
+td class=progress-cell
+  {UIUtils.makeProgressBar(job.numActiveTasks, 
job.numCompletedTasks,
+  job.numFailedTasks, job.numTasks)}
+/td
+  /tr
+}
+
+table class=table table-bordered table-striped table-condensed 
sortable
+  thead{columns}/thead
+  tbody
+{jobs.map(makeRow)}
+  /tbody
+/table
+  }
+
+  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(_.startTime.getOrElse(-1L)).reverse)
+  val completedJobsTable =
+jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse)
+  val failedJobsTable =
+jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse)
+
+  val summary: NodeSeq =
  

[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread andrewor14
Github user andrewor14 commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20677793
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -144,11 +146,30 @@ class JobProgressListener(conf: SparkConf) extends 
SparkListener with Logging {
   }
 
   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)
+// Compute (a potential underestimate of) the number of tasks that 
will be run by this job:
--- End diff --

looks good


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63883863
  
Regarding phantom stages that are skipped:

What do you think about adding a skipped state to visually convey that 
there were stage dependencies that _might_ have been computed but which were 
skipped because their output was already available?  Until a job completes, we 
don't know for sure whether we'll need to go back and re-compute a stage that 
was initially skipped, but after the job has completed the outcome will never 
change.

We could add a Skipped stages section to the job details page.

On the all jobs page, we could display `(x skipped)` in the progress 
bar/cell labels while advancing the bar to 100%, similar to what we do for 
failed tasks.

How does this sound?  Is this more/less confusing than the current behavior?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63896661
  
I don't mind the current behavior, but IMO skipped is better if it's not 
too hard to implement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20685273
  
--- Diff: 
core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala ---
@@ -214,6 +264,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.numActiveStages += 1
+}
--- End diff --

I added a comment over in JobUIData.  Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63897403
  
  [Test build #23687 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23687/consoleFull)
 for   PR 3009 at commit 
[`2bbf41a`](https://github.com/apache/spark/commit/2bbf41a9845c87f0facfb737f26a5b52ba6633ef).
 * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63897673
  
It's not too hard; I'm already done!  :smiley: 

Here's what a job details page looks like when stages were skipped:


![image](https://cloud.githubusercontent.com/assets/50748/5134918/675484ec-70c7-11e4-834e-5b854cc7a2d2.png)

I added some logic to hide tables that are empty, since that significantly 
declutters this page.

Here's what the all jobs page looks like for that same job:


![image](https://cloud.githubusercontent.com/assets/50748/5134932/800fba2e-70c7-11e4-873a-5dd4ddae5c6a.png)

I've still got to implement the JsonProtocol / SparkListenerJobSubmit 
cleanup that Andrew and I discussed, but otherwise I think the user-facing part 
of this PR is ready for final review.  I'll quickly finish up that internal 
cleanup, then remove the [WIP] tag once I'm done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63897855
  
By the way, there are new Selenium tests for these examples of the new 
behavior.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63906253
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/23687/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63906246
  
  [Test build #23687 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23687/consoleFull)
 for   PR 3009 at commit 
[`2bbf41a`](https://github.com/apache/spark/commit/2bbf41a9845c87f0facfb737f26a5b52ba6633ef).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-20 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63908091
  
Alright, I pushed that final cleanup commit.  @andrewor14, want to take a 
final look on the JsonProtocol backwards-compatibility stuff?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-19 Thread kayousterhout
Github user kayousterhout commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63688537
  
There are some DAGScheduler tests that verify that all of the data
structures are empty after jobs run:

https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala#L748.
Can we just do something similar for the UI?  I don't love the size-limited
HashMap idea -- I think we should be able to properly handle these cases
ourselves.

Should I review this again yet, or should I wait for you to fix the phantom
stages thing?

On Tue, Nov 18, 2014 at 11:56 PM, Josh Rosen notificati...@github.com
wrote:

 Just realized that these phantom stages will lead to memory leaks in the
 stageIdToData and stageIdToInfo: we only remove() entries from these data
 structures as a side-effect of removing them from other lists / maps; 
since
 these phantom stages never leave the pending stage, they never enter any
 of the data structures that drive the cleanup process, leading to leaks.

 I also realized that there's also a blocker memory leak in the stable
 progress reporting API: we never actually remove jobs from the jobIdToData
 map. In fact, there's a somewhat embarrassing typo: *retailed* jobs:

   val retailedJobs = conf.getInt(spark.ui.retainedJobs, 
DEFAULT_RETAINED_JOBS)

 This is my fault: I should have added a test that set
 spark.ui.retainedJobs to some low threshold and checked that we only
 reported information for that many jobs. This was sloppy on my part, since
 there was already a corresponding test LRU eviction of stages test that 
I
 should have just copied and modified.

 I wonder if there's a better way to guarantee that we won't leak memory as
 we add more maps / views to JobProgressListener. I think that Guava has
 some size-limited HashMaps with configurable eviction policies; this might
 be safer than relying on the proper cleanup code being called in the right
 place.

 —
 Reply to this email directly or view it on GitHub
 https://github.com/apache/spark/pull/3009#issuecomment-63604040.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-19 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63690828
  
 There are some DAGScheduler tests that verify that all of the data 
structures are empty after jobs run [...] Can we just do something similar for 
the UI?

Sure, there are a few subtleties here that make this trickier.  In 
JobProgressListener, we need to check that our datastructures do not grow 
without bound (not that they're empty), so I think the assertion would look 
something like once I've run at least `spark.ui.retainedStages` stages and 
`spark.ui.retainedJobs` jobs, then the size of JobProgressListener data 
structures that track non-active jobs / stages will not grow and once all 
active jobs / stages complete, then the data structures associated with active 
jobs / stages will be empty.  I guess we also need a third constraint that's 
something like any job / stage that's active will eventually become 
non-active.

I've opened a separate JIRA to fix the existing memory leak so that we can 
review / merge that fix separately (since this PR is already getting pretty 
big).

 Should I review this again yet, or should I wait for you to fix the 
phantom stages thing?

I'd hold off on further review until this afternoon until I finish the 
phantom stages thing.  Once I fix that and the memory leak, then I think this 
will be good to go.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-19 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63753255
  
  [Test build #23655 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23655/consoleFull)
 for   PR 3009 at commit 
[`0b77e3e`](https://github.com/apache/spark/commit/0b77e3e14e9eafc523ac17cc70b4be4766ed0930).
 * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-19 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63755914
  
Alright, I think this should be good to review.  I addressed the memory 
leaks; if you're interested to see whether this works, try deleting any one 
cleanup-related line: it will cause a test failure, showing that the regression 
tests for this work pretty well.

I also added a bunch of Selenium tests as I found bugs / formatting issues, 
so overall I think that this should be much more stable / correct now, 
especially in cases with phantom stages.

There are still a couple of minor wishlist items that I might add if I have 
more time, including additional unit test coverage for some of the new listener 
fields (to make sure that things are _added_ properly, not just removed).  I'd 
also like to maintain a count of the total number of jobs so that we can 
explain that jobs / stages are not being displayed once some have been GC'd.  
There was already a patch to do this for the stages page, so I can just copy 
the logic here.

Finally, it's always nice to add more help text and tooltips.

For 1.2, though, I think this is in good shape and I wouldn't block on 
those minor tasks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-19 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/3009#discussion_r20625666
  
--- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
@@ -269,8 +271,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()
--- End diff --

By the way, I moved this here to avoid a race-condition where a user could 
browse to the web UI before this field was initialized.  There didn't seem to 
be any particular reason for it to be here as opposed to anywhere else.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63758934
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/23655/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-19 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63758931
  
  [Test build #23655 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23655/consoleFull)
 for   PR 3009 at commit 
[`0b77e3e`](https://github.com/apache/spark/commit/0b77e3e14e9eafc523ac17cc70b4be4766ed0930).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-18 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63588777
  
  [Test build #23590 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23590/consoleFull)
 for   PR 3009 at commit 
[`d69c775`](https://github.com/apache/spark/commit/d69c77586d5cea40fc5430d103c5f072ac8ba703).
 * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-18 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63593492
  
  [Test build #23590 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/23590/consoleFull)
 for   PR 3009 at commit 
[`d69c775`](https://github.com/apache/spark/commit/d69c77586d5cea40fc5430d103c5f072ac8ba703).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63593498
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/23590/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-18 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63596404
  
@JoshRosen yeah re: phantom stage ID's. I also noticed this recently in 
relation to something else. It's sort of confusing, but we end up with gaps in 
the ID space.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-11-18 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/3009#issuecomment-63604040
  
Just realized that these phantom stages will lead to memory leaks in the 
`stageIdToData` and `stageIdToInfo`: we only `remove()` entries from these data 
structures as a side-effect of removing them from other lists / maps; since 
these phantom stages never leave the pending stage, they never enter any of 
the data structures that drive the cleanup process, leading to leaks.

I also realized that there's also a blocker memory leak in the stable 
progress reporting API: we never actually remove jobs from the `jobIdToData` 
map.  In fact, there's a somewhat embarrassing typo: _retailed_ jobs:

```scala
  val retailedJobs = conf.getInt(spark.ui.retainedJobs, 
DEFAULT_RETAINED_JOBS)
```

This is my fault: I should have added a test that set 
`spark.ui.retainedJobs` to some low threshold and checked that we only reported 
information for that many jobs.  This was sloppy on my part, since there was 
already a corresponding test LRU eviction of stages test that I should have 
just copied and modified.

I wonder if there's a better way to guarantee that we won't leak memory as 
we add more maps / views to JobProgressListener.  I think that Guava has some 
size-limited HashMaps with configurable eviction policies; this might be safer 
than relying on the proper cleanup code being called in the right place.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request: [SPARK-4145] [WIP] Web UI job pages

2014-10-29 Thread JoshRosen
GitHub user JoshRosen opened a pull request:

https://github.com/apache/spark/pull/3009

[SPARK-4145] [WIP] 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.

### Screenshots

 New UI homepage


![image](https://cloud.githubusercontent.com/assets/50748/4837235/6802f65e-5fd2-11e4-8328-963f5b8b31c5.png)

 Job details page

(This is effectively a per-job version of the stages page that can be 
extended later with other things, such as DAG visualizations)


![image](https://cloud.githubusercontent.com/assets/50748/4837239/79ee6b1e-5fd2-11e4-839d-6cbb42fb01c0.png)


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/JoshRosen/spark job-page

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/3009.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3009


commit 2568a6cfcf5287e29d0e4be58280aa7b934b01a7
Author: Josh Rosen joshro...@databricks.com
Date:   2014-10-29T22:27:16Z

Rename JobProgressPage to AllStagesPage:

This will avoid confusion once we have a page that lists all jobs.

commit 4487dcbe9df550ee054f2ba546ce0553ae40d103
Author: Josh Rosen joshro...@databricks.com
Date:   2014-10-30T00:53:51Z

[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.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org