Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -216,7 +216,7 @@ function renderDagVizForJob(svgContainer) {
var dot = metadata.select(".dot-file").text();
var stageId = metadata.attr("stage-id");
var containerId = VizConstants.graphPrefix + stageId;
var isSkipped = metadata.attr("skipped") == "true";
var isSkipped = metadata.attr("skipped") === "true";
var container;
if (isSkipped) {
container = svgContainer
Expand All @@ -225,11 +225,8 @@ function renderDagVizForJob(svgContainer) {
.attr("skipped", "true");
} else {
// Link each graph to the corresponding stage page (TODO: handle stage attempts)
// Use the link from the stage table so it also works for the history server
var attemptId = 0;
var stageLink = d3.select("#stage-" + stageId + "-" + attemptId)
.select("a.name-link")
.attr("href");
var stageLink = uiRoot + appBasePath + "/stages/stage/?id=" + stageId + "&attempt=" + attemptId;
container = svgContainer
.append("a")
.attr("xlink:href", stageLink)
Expand Down
7 changes: 6 additions & 1 deletion core/src/main/resources/org/apache/spark/ui/static/webui.js
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,16 @@
*/

var uiRoot = "";
var appBasePath = "";

function setUIRoot(val) {
uiRoot = val;
}

function setAppBasePath(path) {
appBasePath = path;
}

function collapseTablePageLoad(name, table){
if (window.localStorage.getItem(name) == "true") {
// Set it to false so that the click function can revert it
Expand All @@ -33,7 +38,7 @@ function collapseTable(thisName, table){
var status = window.localStorage.getItem(thisName) == "true";
status = !status;

var thisClass = '.' + thisName
var thisClass = '.' + thisName;

// Expand the list of additional metrics.
var tableDiv = $(thisClass).parent().find('.' + table);
Expand Down
1 change: 1 addition & 0 deletions core/src/main/scala/org/apache/spark/ui/UIUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -292,6 +292,7 @@ private[spark] object UIUtils extends Logging {
<html>
<head>
{commonHeaderNodes(request)}
<script>setAppBasePath('{activeTab.basePath}')</script>
Copy link
Member

@gengliangwang gengliangwang Jun 2, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can move this line to function commonHeaderNodes, where the method setUIRoot is called.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

At first, I considered the way you mention but to do so, we need to have an additional parameter in commonHeaderNodes to pass activeTab only for this purpose.
Which do you think is better?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see. Let's keep it this way. The method basicSparkPage also calls the function commonHeaderNodes but there is no activeTab in it.

{if (showVisualization) vizHeaderNodes(request) else Seq.empty}
{if (useDataTables) dataTablesHeaderNodes(request) else Seq.empty}
<link rel="shortcut icon"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,28 @@ abstract class RealBrowserUISeleniumSuite(val driverProp: String)
}
}

test("SPARK-31882: Link URL for Stage DAGs should not depend on paged table.") {
withSpark(newSparkContext()) { sc =>
sc.parallelize(1 to 100).map(v => (v, v)).repartition(10).reduceByKey(_ + _).collect

eventually(timeout(10.seconds), interval(50.microseconds)) {
val pathWithPagedTable =
"/jobs/job/?id=0&completedStage.page=2&completedStage.sort=Stage+Id&" +
"completedStage.desc=true&completedStage.pageSize=1#completed"
goToUi(sc, pathWithPagedTable)

// Open DAG Viz.
webDriver.findElement(By.id("job-dag-viz")).click()
val stages = webDriver.findElements(By.cssSelector("svg[class='job'] > a"))
stages.size() should be (3)

stages.get(0).getAttribute("href") should include ("/stages/stage/?id=0&attempt=0")
stages.get(1).getAttribute("href") should include ("/stages/stage/?id=1&attempt=0")
stages.get(2).getAttribute("href") should include ("/stages/stage/?id=2&attempt=0")
}
}
}

/**
* Create a test SparkContext with the SparkUI enabled.
* It is safe to `get` the SparkUI directly from the SparkContext returned here.
Expand Down