Skip to content

Commit f35afa2

Browse files
committed
link application page to master page
1 parent 03d46aa commit f35afa2

File tree

13 files changed

+39
-8
lines changed

13 files changed

+39
-8
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2157,7 +2157,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
21572157
// Note: this code assumes that the task scheduler has been initialized and has contacted
21582158
// the cluster manager to get an application ID (in case the cluster manager provides one).
21592159
listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId),
2160-
startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls))
2160+
startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls,
2161+
schedulerBackend.getMasterWebUiUrl))
21612162
}
21622163

21632164
/** Post the application end event */

core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -127,7 +127,8 @@ private[deploy] object DeployMessages {
127127

128128
// Master to AppClient
129129

130-
case class RegisteredApplication(appId: String, master: RpcEndpointRef) extends DeployMessage
130+
case class RegisteredApplication(appId: String, master: RpcEndpointRef,
131+
masterWebUiUrl: String) extends DeployMessage
131132

132133
// TODO(matei): replace hostPort with host
133134
case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) {
@@ -176,6 +177,7 @@ private[deploy] object DeployMessages {
176177
host: String,
177178
port: Int,
178179
restPort: Option[Int],
180+
webUiUrl: String,
179181
workers: Array[WorkerInfo],
180182
activeApps: Array[ApplicationInfo],
181183
completedApps: Array[ApplicationInfo],

core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -156,7 +156,7 @@ private[spark] class StandaloneAppClient(
156156
}
157157

158158
override def receive: PartialFunction[Any, Unit] = {
159-
case RegisteredApplication(appId_, masterRef) =>
159+
case RegisteredApplication(appId_, masterRef, masterWebUiUrl) =>
160160
// FIXME How to handle the following cases?
161161
// 1. A master receives multiple registrations and sends back multiple
162162
// RegisteredApplications due to an unstable network.
@@ -166,6 +166,7 @@ private[spark] class StandaloneAppClient(
166166
registered.set(true)
167167
master = Some(masterRef)
168168
listener.connected(appId.get)
169+
listener.masterChanged(masterWebUiUrl)
169170

170171
case ApplicationRemoved(message) =>
171172
markDead("Master removed our application: %s".format(message))
@@ -188,6 +189,7 @@ private[spark] class StandaloneAppClient(
188189
case MasterChanged(masterRef, masterWebUiUrl) =>
189190
logInfo("Master has changed, new master is at " + masterRef.address.toSparkURL)
190191
master = Some(masterRef)
192+
listener.masterChanged(masterWebUiUrl)
191193
alreadyDisconnected = false
192194
masterRef.send(MasterChangeAcknowledged(appId.get))
193195
}

core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,4 +37,6 @@ private[spark] trait StandaloneAppClientListener {
3737
fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit
3838

3939
def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit
40+
41+
def masterChanged(masterWebUiUrl: String): Unit
4042
}

core/src/main/scala/org/apache/spark/deploy/master/Master.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -233,7 +233,7 @@ private[deploy] class Master(
233233
registerApplication(app)
234234
logInfo("Registered app " + description.name + " with ID " + app.id)
235235
persistenceEngine.addApplication(app)
236-
driver.send(RegisteredApplication(app.id, self))
236+
driver.send(RegisteredApplication(app.id, self, masterWebUiUrl))
237237
schedule()
238238
}
239239

@@ -471,7 +471,7 @@ private[deploy] class Master(
471471

472472
case RequestMasterState =>
473473
context.reply(MasterStateResponse(
474-
address.host, address.port, restServerBoundPort,
474+
address.host, address.port, restServerBoundPort, masterWebUiUrl,
475475
workers.toArray, apps.toArray, completedApps.toArray,
476476
drivers.toArray, completedDrivers.toArray, state))
477477

core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
3535
def render(request: HttpServletRequest): Seq[Node] = {
3636
val appId = request.getParameter("appId")
3737
val state = master.askWithRetry[MasterStateResponse](RequestMasterState)
38+
val masterWebUiUrl = state.webUiUrl
3839
val app = state.activeApps.find(_.id == appId)
3940
.getOrElse(state.completedApps.find(_.id == appId).orNull)
4041
if (app == null) {
@@ -81,6 +82,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
8182
}
8283
}
8384
</ul>
85+
<p><a href={masterWebUiUrl}>Back to Master</a></p>
8486
</div>
8587
</div>
8688

core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -56,4 +56,6 @@ private[spark] trait SchedulerBackend {
5656
*/
5757
def getDriverLogUrls: Option[Map[String, String]] = None
5858

59+
def getMasterWebUiUrl: Option[String] = None
60+
5961
}

core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -126,7 +126,8 @@ case class SparkListenerApplicationStart(
126126
time: Long,
127127
sparkUser: String,
128128
appAttemptId: Option[String],
129-
driverLogs: Option[Map[String, String]] = None) extends SparkListenerEvent
129+
driverLogs: Option[Map[String, String]] = None,
130+
masterWebUiUrl: Option[String] = None) extends SparkListenerEvent
130131

131132
@DeveloperApi
132133
case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent

core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -157,6 +157,16 @@ private[spark] class StandaloneSchedulerBackend(
157157
removeExecutor(fullId.split("/")(1), reason)
158158
}
159159

160+
var masterWebUiUrl: Option[String] = None
161+
162+
override def masterChanged(webUiUrl: String): Unit = {
163+
masterWebUiUrl = Some(webUiUrl)
164+
}
165+
166+
override def getMasterWebUiUrl(): Option[String] = {
167+
masterWebUiUrl
168+
}
169+
160170
override def sufficientResourcesRegistered(): Boolean = {
161171
totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio
162172
}

core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,10 @@ private[ui] class ExecutorsPage(
5454
private val GCTimePercent = 0.1
5555

5656
def render(request: HttpServletRequest): Seq[Node] = {
57+
val masterWebUiUrl = listener.masterWebUiUrl
58+
val backLink = masterWebUiUrl.map(
59+
link => <p><a href={link}>Back to Master</a></p>
60+
).getOrElse(Seq.empty)
5761
val content =
5862
<div>
5963
{
@@ -63,7 +67,7 @@ private[ui] class ExecutorsPage(
6367
}
6468
</div>;
6569

66-
UIUtils.headerSparkPage("Executors", content, parent, useDataTables = true)
70+
UIUtils.headerSparkPage("Executors", backLink++content, parent, useDataTables = true)
6771
}
6872
}
6973

0 commit comments

Comments
 (0)