aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorjerryshao <sshao@hortonworks.com>2017-01-17 09:30:56 -0600
committerTom Graves <tgraves@yahoo-inc.com>2017-01-17 09:30:56 -0600
commitb79cc7ceb439b3d4e0009963ede3416e3241e562 (patch)
tree7d28019c5144cbb5094edf433fb70e59ffc121e6
parent6c00c069e3c3f5904abd122cea1d56683031cca0 (diff)
downloadspark-b79cc7ceb439b3d4e0009963ede3416e3241e562.tar.gz
spark-b79cc7ceb439b3d4e0009963ede3416e3241e562.tar.bz2
spark-b79cc7ceb439b3d4e0009963ede3416e3241e562.zip
[SPARK-19179][YARN] Change spark.yarn.access.namenodes config and update docs
## What changes were proposed in this pull request? `spark.yarn.access.namenodes` configuration cannot actually reflects the usage of it, inside the code it is the Hadoop filesystems we get tokens, not NNs. So here propose to update the name of this configuration, also change the related code and doc. ## How was this patch tested? Local verification. Author: jerryshao <sshao@hortonworks.com> Closes #16560 from jerryshao/SPARK-19179.
-rw-r--r--core/src/main/scala/org/apache/spark/SparkConf.scala6
-rw-r--r--docs/running-on-yarn.md19
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala5
-rw-r--r--resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala8
4 files changed, 23 insertions, 15 deletions
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index d78b9f1b29..601d24191e 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -699,8 +699,10 @@ private[spark] object SparkConf extends Logging {
"spark.rpc.message.maxSize" -> Seq(
AlternateConfig("spark.akka.frameSize", "1.6")),
"spark.yarn.jars" -> Seq(
- AlternateConfig("spark.yarn.jar", "2.0"))
- )
+ AlternateConfig("spark.yarn.jar", "2.0")),
+ "spark.yarn.access.hadoopFileSystems" -> Seq(
+ AlternateConfig("spark.yarn.access.namenodes", "2.2"))
+ )
/**
* A view of `configsWithAlternatives` that makes it more efficient to look up deprecated
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index f7513454c7..051f64e1be 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -276,15 +276,16 @@ To use a custom metrics.properties for the application master and executors, upd
</td>
</tr>
<tr>
- <td><code>spark.yarn.access.namenodes</code></td>
+ <td><code>spark.yarn.access.hadoopFileSystems</code></td>
<td>(none)</td>
<td>
- A comma-separated list of secure HDFS namenodes your Spark application is going to access. For
- example, <code>spark.yarn.access.namenodes=hdfs://nn1.com:8032,hdfs://nn2.com:8032,
- webhdfs://nn3.com:50070</code>. The Spark application must have access to the namenodes listed
+ A comma-separated list of secure Hadoop filesystems your Spark application is going to access. For
+ example, <code>spark.yarn.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032,
+ webhdfs://nn3.com:50070</code>. The Spark application must have access to the filesystems listed
and Kerberos must be properly configured to be able to access them (either in the same realm
- or in a trusted realm). Spark acquires security tokens for each of the namenodes so that
- the Spark application can access those remote HDFS clusters.
+ or in a trusted realm). Spark acquires security tokens for each of the filesystems so that
+ the Spark application can access those remote Hadoop filesystems. <code>spark.yarn.access.namenodes</code>
+ is deprecated, please use this instead.
</td>
</tr>
<tr>
@@ -496,10 +497,10 @@ includes a URI of the metadata store in `"hive.metastore.uris`, and
If an application needs to interact with other secure Hadoop filesystems, then
the tokens needed to access these clusters must be explicitly requested at
-launch time. This is done by listing them in the `spark.yarn.access.namenodes` property.
+launch time. This is done by listing them in the `spark.yarn.access.hadoopFileSystems` property.
```
-spark.yarn.access.namenodes hdfs://ireland.example.org:8020/,webhdfs://frankfurt.example.org:50070/
+spark.yarn.access.hadoopFileSystems hdfs://ireland.example.org:8020/,webhdfs://frankfurt.example.org:50070/
```
Spark supports integrating with other security-aware services through Java Services mechanism (see
@@ -574,7 +575,7 @@ spark.yarn.security.credentials.hive.enabled false
spark.yarn.security.credentials.hbase.enabled false
```
-The configuration option `spark.yarn.access.namenodes` must be unset.
+The configuration option `spark.yarn.access.hadoopFileSystems` must be unset.
## Troubleshooting Kerberos
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
index 666cb456a9..f19a5b22a7 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
@@ -248,6 +248,11 @@ package object config {
.toSequence
.createWithDefault(Nil)
+ private[spark] val FILESYSTEMS_TO_ACCESS = ConfigBuilder("spark.yarn.access.hadoopFileSystems")
+ .doc("Extra Hadoop filesystem URLs for which to request delegation tokens. The filesystem " +
+ "that hosts fs.defaultFS does not need to be listed here.")
+ .fallbackConf(NAMENODES_TO_ACCESS)
+
/* Rolled log aggregation configuration. */
private[spark] val ROLLED_LOG_INCLUDE_PATTERN =
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala
index b4fb4a790a..f65c886db9 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala
@@ -47,7 +47,7 @@ private[security] class HadoopFSCredentialProvider
// NameNode to access, used to get tokens from different FileSystems
val tmpCreds = new Credentials()
val tokenRenewer = getTokenRenewer(hadoopConf)
- nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
+ hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst =>
val dstFs = dst.getFileSystem(hadoopConf)
logInfo("getting token for: " + dst)
dstFs.addDelegationTokens(tokenRenewer, tmpCreds)
@@ -80,7 +80,7 @@ private[security] class HadoopFSCredentialProvider
// user as renewer.
sparkConf.get(PRINCIPAL).flatMap { renewer =>
val creds = new Credentials()
- nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
+ hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst =>
val dstFs = dst.getFileSystem(hadoopConf)
dstFs.addDelegationTokens(renewer, creds)
}
@@ -112,8 +112,8 @@ private[security] class HadoopFSCredentialProvider
delegTokenRenewer
}
- private def nnsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = {
- sparkConf.get(NAMENODES_TO_ACCESS).map(new Path(_)).toSet +
+ private def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = {
+ sparkConf.get(FILESYSTEMS_TO_ACCESS).map(new Path(_)).toSet +
sparkConf.get(STAGING_DIR).map(new Path(_))
.getOrElse(FileSystem.get(hadoopConf).getHomeDirectory)
}