aboutsummaryrefslogtreecommitdiff
path: root/core/src/main
diff options
context:
space:
mode:
Diffstat (limited to 'core/src/main')
-rw-r--r--core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js4
-rw-r--r--core/src/main/scala/org/apache/spark/ui/UIUtils.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala12
3 files changed, 23 insertions, 3 deletions
diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js
index 76eb2c4641..cda27cad7e 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js
@@ -178,8 +178,8 @@ function renderDagVizForJob(svgContainer) {
var stageId = metadata.attr("stage-id");
var containerId = VizConstants.graphPrefix + stageId;
// Link each graph to the corresponding stage page (TODO: handle stage attempts)
- var stageLink =
- "/stages/stage/?id=" + stageId.replace(VizConstants.stagePrefix, "") + "&attempt=0";
+ var stageLink = "/stages/stage/?id=" +
+ stageId.replace(VizConstants.stagePrefix, "") + "&attempt=0&expandDagViz=true";
var container = svgContainer
.append("a")
.attr("xlink:href", stageLink)
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 e2d03f8342..97eed13c2d 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -20,7 +20,7 @@ package org.apache.spark.ui
import java.text.SimpleDateFormat
import java.util.{Locale, Date}
-import scala.xml.{Node, Text}
+import scala.xml.{Node, Text, Unparsed}
import org.apache.spark.Logging
import org.apache.spark.ui.scope.RDDOperationGraph
@@ -371,4 +371,12 @@ private[spark] object UIUtils extends Logging {
</div>
</div>
}
+
+ /** Return a script element that automatically expands the DAG visualization on page load. */
+ def expandDagVizOnLoad(forJob: Boolean): Seq[Node] = {
+ <script type="text/javascript">
+ {Unparsed("$(document).ready(function() { toggleDagViz(" + forJob + ") });")}
+ </script>
+ }
+
}
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 579310070c..6c4305873c 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
@@ -44,6 +44,10 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
val parameterAttempt = request.getParameter("attempt")
require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter")
+ // If this is set, expand the dag visualization by default
+ val expandDagVizParam = request.getParameter("expandDagViz")
+ val expandDagViz = expandDagVizParam != null && expandDagVizParam.toBoolean
+
val stageId = parameterId.toInt
val stageAttemptId = parameterAttempt.toInt
val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId))
@@ -174,6 +178,13 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
val dagViz = UIUtils.showDagVizForStage(
stageId, operationGraphListener.getOperationGraphForStage(stageId))
+ val maybeExpandDagViz: Seq[Node] =
+ if (expandDagViz) {
+ UIUtils.expandDagVizOnLoad(forJob = false)
+ } else {
+ Seq.empty
+ }
+
val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value")
def accumulableRow(acc: AccumulableInfo): Elem =
<tr><td>{acc.name}</td><td>{acc.value}</td></tr>
@@ -440,6 +451,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
summary ++
showAdditionalMetrics ++
dagViz ++
+ maybeExpandDagViz ++
<h4>Summary Metrics for {numCompleted} Completed Tasks</h4> ++
<div>{summaryTable.getOrElse("No tasks have reported metrics yet.")}</div> ++
<h4>Aggregated Metrics by Executor</h4> ++ executorTable.toNodeSeq ++