Skip to content

Commit 46e224a

Browse files
Rahul Singhaltgravescs
authored andcommitted
SPARK-2150: Provide direct link to finished application UI in yarn resou...
...rce manager UI Use the event logger directory to provide a direct link to finished application UI in yarn resourcemanager UI. Author: Rahul Singhal <[email protected]> Closes #1094 from rahulsinghaliitd/SPARK-2150 and squashes the following commits: 95f230c [Rahul Singhal] SPARK-2150: Provide direct link to finished application UI in yarn resource manager UI
1 parent 42dfab7 commit 46e224a

File tree

11 files changed

+46
-13
lines changed

11 files changed

+46
-13
lines changed

core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -169,7 +169,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
169169
val ui: SparkUI = if (renderUI) {
170170
val conf = this.conf.clone()
171171
val appSecManager = new SecurityManager(conf)
172-
new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId)
172+
new SparkUI(conf, appSecManager, replayBus, appId,
173+
HistoryServer.UI_PATH_PREFIX + s"/$appId")
173174
// Do not call ui.bind() to avoid creating a new server for each application
174175
} else {
175176
null

core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
7575
"Last Updated")
7676

7777
private def appRow(info: ApplicationHistoryInfo): Seq[Node] = {
78-
val uiAddress = "/history/" + info.id
78+
val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}"
7979
val startTime = UIUtils.formatDate(info.startTime)
8080
val endTime = UIUtils.formatDate(info.endTime)
8181
val duration = UIUtils.formatDuration(info.endTime - info.startTime)

core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -114,7 +114,7 @@ class HistoryServer(
114114
attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
115115

116116
val contextHandler = new ServletContextHandler
117-
contextHandler.setContextPath("/history")
117+
contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX)
118118
contextHandler.addServlet(new ServletHolder(loaderServlet), "/*")
119119
attachHandler(contextHandler)
120120
}
@@ -172,6 +172,8 @@ class HistoryServer(
172172
object HistoryServer extends Logging {
173173
private val conf = new SparkConf
174174

175+
val UI_PATH_PREFIX = "/history"
176+
175177
def main(argStrings: Array[String]) {
176178
SignalLogger.register(log)
177179
initSecurity()

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

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import akka.serialization.SerializationExtension
3535
import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException}
3636
import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState}
3737
import org.apache.spark.deploy.DeployMessages._
38+
import org.apache.spark.deploy.history.HistoryServer
3839
import org.apache.spark.deploy.master.DriverState.DriverState
3940
import org.apache.spark.deploy.master.MasterMessages._
4041
import org.apache.spark.deploy.master.ui.MasterWebUI
@@ -664,9 +665,10 @@ private[spark] class Master(
664665
*/
665666
def rebuildSparkUI(app: ApplicationInfo): Boolean = {
666667
val appName = app.desc.name
668+
val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found"
667669
val eventLogDir = app.desc.eventLogDir.getOrElse {
668670
// Event logging is not enabled for this application
669-
app.desc.appUiUrl = "/history/not-found"
671+
app.desc.appUiUrl = notFoundBasePath
670672
return false
671673
}
672674
val fileSystem = Utils.getHadoopFileSystem(eventLogDir)
@@ -681,13 +683,14 @@ private[spark] class Master(
681683
logWarning(msg)
682684
msg += " Did you specify the correct logging directory?"
683685
msg = URLEncoder.encode(msg, "UTF-8")
684-
app.desc.appUiUrl = s"/history/not-found?msg=$msg&title=$title"
686+
app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title"
685687
return false
686688
}
687689

688690
try {
689691
val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec)
690-
val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id)
692+
val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)",
693+
HistoryServer.UI_PATH_PREFIX + s"/${app.id}")
691694
replayBus.replay()
692695
appIdToUI(app.id) = ui
693696
webUi.attachSparkUI(ui)
@@ -702,7 +705,7 @@ private[spark] class Master(
702705
var msg = s"Exception in replaying log for application $appName!"
703706
logError(msg, e)
704707
msg = URLEncoder.encode(msg, "UTF-8")
705-
app.desc.appUiUrl = s"/history/not-found?msg=$msg&exception=$exception&title=$title"
708+
app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title"
706709
false
707710
}
708711
}

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

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -63,6 +63,13 @@ private[spark] class EventLoggingListener(
6363
// For testing. Keep track of all JSON serialized events that have been logged.
6464
private[scheduler] val loggedEvents = new ArrayBuffer[JValue]
6565

66+
/**
67+
* Return only the unique application directory without the base directory.
68+
*/
69+
def getApplicationLogDir(): String = {
70+
name
71+
}
72+
6673
/**
6774
* Begin logging events.
6875
* If compression is used, log a file that indicates which compression library is used.

yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
6060
private var yarnAllocator: YarnAllocationHandler = _
6161
private var isFinished: Boolean = false
6262
private var uiAddress: String = _
63+
private var uiHistoryAddress: String = _
6364
private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
6465
YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES)
6566
private var isLastAMRetry: Boolean = true
@@ -237,6 +238,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
237238

238239
if (null != sparkContext) {
239240
uiAddress = sparkContext.ui.appUIHostPort
241+
uiHistoryAddress = YarnSparkHadoopUtil.getUIHistoryAddress(sparkContext, sparkConf)
240242
this.yarnAllocator = YarnAllocationHandler.newAllocator(
241243
yarnConf,
242244
resourceManager,
@@ -360,7 +362,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
360362
finishReq.setAppAttemptId(appAttemptId)
361363
finishReq.setFinishApplicationStatus(status)
362364
finishReq.setDiagnostics(diagnostics)
363-
finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", ""))
365+
finishReq.setTrackingUrl(uiHistoryAddress)
364366
resourceManager.finishApplicationMaster(finishReq)
365367
}
366368
}

yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -289,7 +289,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp
289289
.asInstanceOf[FinishApplicationMasterRequest]
290290
finishReq.setAppAttemptId(appAttemptId)
291291
finishReq.setFinishApplicationStatus(status)
292-
finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", ""))
292+
finishReq.setTrackingUrl(sparkConf.get("spark.driver.appUIHistoryAddress", ""))
293293
resourceManager.finishApplicationMaster(finishReq)
294294
}
295295

yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,9 @@ import org.apache.hadoop.util.StringInterner
3030
import org.apache.hadoop.yarn.conf.YarnConfiguration
3131
import org.apache.hadoop.yarn.api.ApplicationConstants
3232
import org.apache.hadoop.conf.Configuration
33+
34+
import org.apache.spark.{SparkConf, SparkContext}
35+
import org.apache.spark.deploy.history.HistoryServer
3336
import org.apache.spark.deploy.SparkHadoopUtil
3437

3538
/**
@@ -132,4 +135,17 @@ object YarnSparkHadoopUtil {
132135
}
133136
}
134137

138+
def getUIHistoryAddress(sc: SparkContext, conf: SparkConf) : String = {
139+
val eventLogDir = sc.eventLogger match {
140+
case Some(logger) => logger.getApplicationLogDir()
141+
case None => ""
142+
}
143+
val historyServerAddress = conf.get("spark.yarn.historyServer.address", "")
144+
if (historyServerAddress != "" && eventLogDir != "") {
145+
historyServerAddress + HistoryServer.UI_PATH_PREFIX + s"/$eventLogDir"
146+
} else {
147+
""
148+
}
149+
}
150+
135151
}

yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster
1919

2020
import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState}
2121
import org.apache.spark.{SparkException, Logging, SparkContext}
22-
import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher}
22+
import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher, YarnSparkHadoopUtil}
2323
import org.apache.spark.scheduler.TaskSchedulerImpl
2424

2525
import scala.collection.mutable.ArrayBuffer
@@ -54,6 +54,7 @@ private[spark] class YarnClientSchedulerBackend(
5454
val driverPort = conf.get("spark.driver.port")
5555
val hostport = driverHost + ":" + driverPort
5656
conf.set("spark.driver.appUIAddress", sc.ui.appUIHostPort)
57+
conf.set("spark.driver.appUIHistoryAddress", YarnSparkHadoopUtil.getUIHistoryAddress(sc, conf))
5758

5859
val argsArrayBuf = new ArrayBuffer[String]()
5960
argsArrayBuf += (

yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
5959
private var yarnAllocator: YarnAllocationHandler = _
6060
private var isFinished: Boolean = false
6161
private var uiAddress: String = _
62+
private var uiHistoryAddress: String = _
6263
private val maxAppAttempts: Int = conf.getInt(
6364
YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
6465
private var isLastAMRetry: Boolean = true
@@ -216,6 +217,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
216217

217218
if (sparkContext != null) {
218219
uiAddress = sparkContext.ui.appUIHostPort
220+
uiHistoryAddress = YarnSparkHadoopUtil.getUIHistoryAddress(sparkContext, sparkConf)
219221
this.yarnAllocator = YarnAllocationHandler.newAllocator(
220222
yarnConf,
221223
amClient,
@@ -312,8 +314,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
312314

313315
logInfo("Unregistering ApplicationMaster with " + status)
314316
if (registered) {
315-
val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "")
316-
amClient.unregisterApplicationMaster(status, diagnostics, trackingUrl)
317+
amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress)
317318
}
318319
}
319320
}

0 commit comments

Comments
 (0)