Skip to content
Snippets Groups Projects
Commit 950645d5 authored by Dean Chen's avatar Dean Chen Committed by Sean Owen
Browse files

[SPARK-6868][YARN] Fix broken container log link on executor page when HTTPS_ONLY.

Correct http schema in YARN container log link in Spark UI when container logs when YARN is configured to be HTTPS_ONLY.

Uses the same logic as the YARN jobtracker webapp. Entry point is [JobBlock](https://github.com/apache/hadoop/blob/e1109fb65608a668cd53dc324dadc6f63a74eeb9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java#L108) and logic is in [MRWebAppUtil](https://github.com/apache/hadoop/blob/e1109fb65608a668cd53dc324dadc6f63a74eeb9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java#L75).

I chose to migrate the logic over instead of importing MRWebAppUtil(but can update the PR to do so) since the class is designated as private and the logic was straightforward.

Author: Dean Chen <deanchen5@gmail.com>

Closes #5477 from deanchen/container-url and squashes the following commits:

91d3090 [Dean Chen] Correct http schema in YARN container log link in Spark UI when container logs when YARN is configured to be HTTPS_ONLY.
parent 68d1faa3
No related branches found
No related tags found
No related merge requests found
...@@ -290,10 +290,19 @@ class ExecutorRunnable( ...@@ -290,10 +290,19 @@ class ExecutorRunnable(
YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs)
} }
// lookup appropriate http scheme for container log urls
val yarnHttpPolicy = yarnConf.get(
YarnConfiguration.YARN_HTTP_POLICY_KEY,
YarnConfiguration.YARN_HTTP_POLICY_DEFAULT
)
val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://"
// Add log urls // Add log urls
sys.env.get("SPARK_USER").foreach { user => sys.env.get("SPARK_USER").foreach { user =>
val baseUrl = "http://%s/node/containerlogs/%s/%s" val containerId = ConverterUtils.toString(container.getId)
.format(container.getNodeHttpAddress, ConverterUtils.toString(container.getId), user) val address = container.getNodeHttpAddress
val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user"
env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=0" env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=0"
env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=0" env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=0"
} }
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment