Skip to content

Commit

Permalink
[SPARK-20044][UI] Support Spark UI behind front-end reverse proxy usi…
Browse files Browse the repository at this point in the history
…ng a path prefix Revert proxy url

### What changes were proposed in this pull request?

Allow to run the Spark web UI behind a reverse proxy with URLs prefixed by a context root, like www.mydomain.com/spark. In particular, this allows to access multiple Spark clusters through the same virtual host, only distinguishing them by context root, like www.mydomain.com/cluster1, www.mydomain.com/cluster2, and it allows to run the Spark UI in a common cookie domain (for SSO) with other services.

### Why are the changes needed?

This PR is to take over #17455.
After changes, Spark allows showing customized prefix URL in all the `href` links of the HTML pages.

### Does this PR introduce _any_ user-facing change?

Yes, all the links of UI pages will be contains the value of `spark.ui.reverseProxyUrl` if it is configurated.
### How was this patch tested?

New HTML Unit tests in MasterSuite
Manual UI testing for master, worker and app UI with an nginx proxy
Spark config:
```
spark.ui.port 8080
spark.ui.reverseProxy=true
spark.ui.reverseProxyUrl=/path/to/spark/
```
nginx config:
```
server {
    listen 9000;
    set $SPARK_MASTER http://127.0.0.1:8080;
    # split spark UI path into prefix and local path within master UI
    location ~ ^(/path/to/spark/) {
        # strip prefix when forwarding request
        rewrite /path/to/spark(/.*) $1  break;
        #rewrite /path/to/spark/ "/" ;
        # forward to spark master UI
        proxy_pass $SPARK_MASTER;
        proxy_intercept_errors on;
        error_page 301 302 307 = handle_redirects;
    }
    location handle_redirects {
        set $saved_redirect_location '$upstream_http_location';
        proxy_pass $saved_redirect_location;
    }
}
```

Closes #29820 from gengliangwang/revertProxyURL.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Oliver Köth <okoeth@de.ibm.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
  • Loading branch information
gengliangwang and okoethibm committed Nov 1, 2020
1 parent b8a440f commit 2b6dfa5
Show file tree
Hide file tree
Showing 7 changed files with 140 additions and 13 deletions.
4 changes: 3 additions & 1 deletion core/src/main/scala/org/apache/spark/SparkContext.scala
Expand Up @@ -570,7 +570,9 @@ class SparkContext(config: SparkConf) extends Logging {
_applicationAttemptId = _taskScheduler.applicationAttemptId()
_conf.set("spark.app.id", _applicationId)
if (_conf.get(UI_REVERSE_PROXY)) {
System.setProperty("spark.ui.proxyBase", "/proxy/" + _applicationId)
val proxyUrl = _conf.get(UI_REVERSE_PROXY_URL.key, "").stripSuffix("/") +
"/proxy/" + _applicationId
System.setProperty("spark.ui.proxyBase", proxyUrl)
}
_ui.foreach(_.setAppId(_applicationId))
_env.blockManager.initialize(_applicationId)
Expand Down
Expand Up @@ -147,7 +147,13 @@ private[deploy] class Master(
webUi.bind()
masterWebUiUrl = s"${webUi.scheme}$masterPublicAddress:${webUi.boundPort}"
if (reverseProxy) {
masterWebUiUrl = conf.get(UI_REVERSE_PROXY_URL).orElse(Some(masterWebUiUrl)).get
val uiReverseProxyUrl = conf.get(UI_REVERSE_PROXY_URL).map(_.stripSuffix("/"))
if (uiReverseProxyUrl.nonEmpty) {
System.setProperty("spark.ui.proxyBase", uiReverseProxyUrl.get)
// If the master URL has a path component, it must end with a slash.
// Otherwise the browser generates incorrect relative links
masterWebUiUrl = uiReverseProxyUrl.get + "/"
}
webUi.addProxy()
logInfo(s"Spark Master is acting as a reverse proxy. Master, Workers and " +
s"Applications UIs are available at $masterWebUiUrl")
Expand Down
Expand Up @@ -171,7 +171,8 @@ private[deploy] class ExecutorRunner(
// Add webUI log urls
val baseUrl =
if (conf.get(UI_REVERSE_PROXY)) {
s"/proxy/$workerId/logPage/?appId=$appId&executorId=$execId&logType="
conf.get(UI_REVERSE_PROXY_URL.key, "").stripSuffix("/") +
s"/proxy/$workerId/logPage/?appId=$appId&executorId=$execId&logType="
} else {
s"$webUiScheme$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType="
}
Expand Down
Expand Up @@ -276,7 +276,14 @@ private[deploy] class Worker(
master = Some(masterRef)
connected = true
if (reverseProxy) {
logInfo(s"WorkerWebUI is available at $activeMasterWebUiUrl/proxy/$workerId")
logInfo("WorkerWebUI is available at %s/proxy/%s".format(
activeMasterWebUiUrl.stripSuffix("/"), workerId))
// if reverseProxyUrl is not set, then we continue to generate relative URLs
// starting with "/" throughout the UI and do not use activeMasterWebUiUrl
val proxyUrl = conf.get(UI_REVERSE_PROXY_URL.key, "").stripSuffix("/")
// In the method `UIUtils.makeHref`, the URL segment "/proxy/$worker_id" will be appended
// after `proxyUrl`, so no need to set the worker ID in the `spark.ui.proxyBase` here.
System.setProperty("spark.ui.proxyBase", proxyUrl)
}
// Cancel any outstanding re-registration attempts because we found a new master
cancelLastRegistrationRetry()
Expand Down
3 changes: 2 additions & 1 deletion core/src/main/scala/org/apache/spark/ui/UIUtils.scala
Expand Up @@ -639,7 +639,8 @@ private[spark] object UIUtils extends Logging {
*/
def makeHref(proxy: Boolean, id: String, origHref: String): String = {
if (proxy) {
s"/proxy/$id"
val proxyPrefix = sys.props.getOrElse("spark.ui.proxyBase", "")
proxyPrefix + "/proxy/" + id
} else {
origHref
}
Expand Down
101 changes: 95 additions & 6 deletions core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala
Expand Up @@ -143,6 +143,10 @@ class MockExecutorLaunchFailWorker(master: Master, conf: SparkConf = new SparkCo
class MasterSuite extends SparkFunSuite
with Matchers with Eventually with PrivateMethodTester with BeforeAndAfter {

// regex to extract worker links from the master webui HTML
// groups represent URL and worker ID
val WORKER_LINK_RE = """<a href="(.+?)">\s*(worker-.+?)\s*</a>""".r

private var _master: Master = _

after {
Expand Down Expand Up @@ -320,10 +324,10 @@ class MasterSuite extends SparkFunSuite
val conf = new SparkConf()
val localCluster = new LocalSparkCluster(2, 2, 512, conf)
localCluster.start()
val masterUrl = s"http://localhost:${localCluster.masterWebUIPort}"
try {
eventually(timeout(5.seconds), interval(100.milliseconds)) {
val json = Source.fromURL(s"http://localhost:${localCluster.masterWebUIPort}/json")
.getLines().mkString("\n")
val json = Source.fromURL(s"$masterUrl/json").getLines().mkString("\n")
val JArray(workers) = (parse(json) \ "workers")
workers.size should be (2)
workers.foreach { workerSummaryJson =>
Expand All @@ -332,6 +336,16 @@ class MasterSuite extends SparkFunSuite
.getLines().mkString("\n"))
(workerResponse \ "cores").extract[Int] should be (2)
}

val html = Source.fromURL(s"$masterUrl/").getLines().mkString("\n")
html should include ("Spark Master at spark://")
val workerLinks = (WORKER_LINK_RE findAllMatchIn html).toList
workerLinks.size should be (2)
workerLinks foreach { case WORKER_LINK_RE(workerUrl, workerId) =>
val workerHtml = Source.fromURL(workerUrl).getLines().mkString("\n")
workerHtml should include ("Spark Worker at")
workerHtml should include ("Running Executors (0)")
}
}
} finally {
localCluster.stop()
Expand All @@ -340,31 +354,106 @@ class MasterSuite extends SparkFunSuite

test("master/worker web ui available with reverseProxy") {
implicit val formats = org.json4s.DefaultFormats
val reverseProxyUrl = "http://localhost:8080"
val conf = new SparkConf()
conf.set(UI_REVERSE_PROXY, true)
val localCluster = new LocalSparkCluster(2, 2, 512, conf)
localCluster.start()
val masterUrl = s"http://localhost:${localCluster.masterWebUIPort}"
try {
eventually(timeout(5.seconds), interval(100.milliseconds)) {
val json = Source.fromURL(s"$masterUrl/json")
.getLines().mkString("\n")
val JArray(workers) = (parse(json) \ "workers")
workers.size should be (2)
workers.foreach { workerSummaryJson =>
// the webuiaddress intentionally points to the local web ui.
// explicitly construct reverse proxy url targeting the master
val JString(workerId) = workerSummaryJson \ "id"
val url = s"$masterUrl/proxy/${workerId}/json"
val workerResponse = parse(Source.fromURL(url).getLines().mkString("\n"))
(workerResponse \ "cores").extract[Int] should be (2)
}

val html = Source.fromURL(s"$masterUrl/").getLines().mkString("\n")
html should include ("Spark Master at spark://")
html should include ("""href="/static""")
html should include ("""src="/static""")
verifyWorkerUI(html, masterUrl)
}
} finally {
localCluster.stop()
System.getProperties().remove("spark.ui.proxyBase")
}
}

test("master/worker web ui available behind front-end reverseProxy") {
implicit val formats = org.json4s.DefaultFormats
val reverseProxyUrl = "http://proxyhost:8080/path/to/spark"
val conf = new SparkConf()
conf.set(UI_REVERSE_PROXY, true)
conf.set(UI_REVERSE_PROXY_URL, reverseProxyUrl)
val localCluster = new LocalSparkCluster(2, 2, 512, conf)
localCluster.start()
val masterUrl = s"http://localhost:${localCluster.masterWebUIPort}"
try {
eventually(timeout(5.seconds), interval(100.milliseconds)) {
val json = Source.fromURL(s"http://localhost:${localCluster.masterWebUIPort}/json")
val json = Source.fromURL(s"$masterUrl/json")
.getLines().mkString("\n")
val JArray(workers) = (parse(json) \ "workers")
workers.size should be (2)
workers.foreach { workerSummaryJson =>
// the webuiaddress intentionally points to the local web ui.
// explicitly construct reverse proxy url targeting the master
val JString(workerId) = workerSummaryJson \ "id"
val url = s"http://localhost:${localCluster.masterWebUIPort}/proxy/${workerId}/json"
val url = s"$masterUrl/proxy/${workerId}/json"
val workerResponse = parse(Source.fromURL(url).getLines().mkString("\n"))
(workerResponse \ "cores").extract[Int] should be (2)
(workerResponse \ "masterwebuiurl").extract[String] should be (reverseProxyUrl)
(workerResponse \ "masterwebuiurl").extract[String] should be (reverseProxyUrl + "/")
}

// with LocalCluster, we have masters and workers in the same JVM, each overwriting
// system property spark.ui.proxyBase.
// so we need to manage this property explicitly for test
System.getProperty("spark.ui.proxyBase") should startWith
(s"$reverseProxyUrl/proxy/worker-")
System.setProperty("spark.ui.proxyBase", reverseProxyUrl)
val html = Source.fromURL(s"$masterUrl/").getLines().mkString("\n")
html should include ("Spark Master at spark://")
verifyStaticResourcesServedByProxy(html, reverseProxyUrl)
verifyWorkerUI(html, masterUrl, reverseProxyUrl)
}
} finally {
localCluster.stop()
System.getProperties().remove("spark.ui.proxyBase")
}
}

private def verifyWorkerUI(masterHtml: String, masterUrl: String,
reverseProxyUrl: String = ""): Unit = {
val workerLinks = (WORKER_LINK_RE findAllMatchIn masterHtml).toList
workerLinks.size should be (2)
workerLinks foreach {
case WORKER_LINK_RE(workerUrl, workerId) =>
workerUrl should be (s"$reverseProxyUrl/proxy/$workerId")
// there is no real front-end proxy as defined in $reverseProxyUrl
// construct url directly targeting the master
val url = s"$masterUrl/proxy/$workerId/"
System.setProperty("spark.ui.proxyBase", workerUrl)
val workerHtml = Source.fromURL(url).getLines().mkString("\n")
workerHtml should include ("Spark Worker at")
workerHtml should include ("Running Executors (0)")
verifyStaticResourcesServedByProxy(workerHtml, workerUrl)
case _ => fail // make sure we don't accidentially skip the tests
}
}

private def verifyStaticResourcesServedByProxy(html: String, proxyUrl: String): Unit = {
html should not include ("""href="/static""")
html should include (s"""href="$proxyUrl/static""")
html should not include ("""src="/static""")
html should include (s"""src="$proxyUrl/static""")
}

test("basic scheduling - spread out") {
basicScheduling(spreadOut = true)
}
Expand Down
25 changes: 23 additions & 2 deletions docs/configuration.md
Expand Up @@ -1193,8 +1193,29 @@ Apart from these, the following properties are also available, and may be useful
<td><code>spark.ui.reverseProxyUrl</code></td>
<td></td>
<td>
This is the URL where your proxy is running. This URL is for proxy which is running in front of Spark Master. This is useful when running proxy for authentication e.g. OAuth proxy. Make sure this is a complete URL including scheme (http/https) and port to reach your proxy.
</td>
If the Spark UI should be served through another front-end reverse proxy, this is the URL
for accessing the Spark master UI through that reverse proxy.
This is useful when running proxy for authentication e.g. an OAuth proxy. The URL may contain
a path prefix, like <code>http://mydomain.com/path/to/spark/</code>, allowing you to serve the
UI for multiple Spark clusters and other web applications through the same virtual host and
port.
Normally, this should be an absolute URL including scheme (http/https), host and port.
It is possible to specify a relative URL starting with "/" here. In this case, all URLs
generated by the Spark UI and Spark REST APIs will be server-relative links -- this will still
work, as the entire Spark UI is served through the same host and port.
<br/>The setting affects link generation in the Spark UI, but the front-end reverse proxy
is responsible for
<ul>
<li>stripping a path prefix before forwarding the request,</li>
<li>rewriting redirects which point directly to the Spark master,</li>
<li>redirecting access from <code>http://mydomain.com/path/to/spark</code> to
<code>http://mydomain.com/path/to/spark/</code> (trailing slash after path prefix); otherwise
relative links on the master page do not work correctly.</li>
</ul>
This setting affects all the workers and application UIs running in the cluster and must be set
identically on all the workers, drivers and masters. In is only effective when
<code>spark.ui.reverseProxy</code> is turned on. This setting is not needed when the Spark
master web UI is directly reachable. </td>
<td>2.1.0</td>
</tr>
<tr>
Expand Down

0 comments on commit 2b6dfa5

Please sign in to comment.