Skip to content

Commit b79cc7c

Browse files
jerryshaoTom Graves
authored andcommitted
[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 <[email protected]> Closes #16560 from jerryshao/SPARK-19179.
1 parent 6c00c06 commit b79cc7c

File tree

4 files changed

+23
-15
lines changed

4 files changed

+23
-15
lines changed

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

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -699,8 +699,10 @@ private[spark] object SparkConf extends Logging {
699699
"spark.rpc.message.maxSize" -> Seq(
700700
AlternateConfig("spark.akka.frameSize", "1.6")),
701701
"spark.yarn.jars" -> Seq(
702-
AlternateConfig("spark.yarn.jar", "2.0"))
703-
)
702+
AlternateConfig("spark.yarn.jar", "2.0")),
703+
"spark.yarn.access.hadoopFileSystems" -> Seq(
704+
AlternateConfig("spark.yarn.access.namenodes", "2.2"))
705+
)
704706

705707
/**
706708
* A view of `configsWithAlternatives` that makes it more efficient to look up deprecated

docs/running-on-yarn.md

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -276,15 +276,16 @@ To use a custom metrics.properties for the application master and executors, upd
276276
</td>
277277
</tr>
278278
<tr>
279-
<td><code>spark.yarn.access.namenodes</code></td>
279+
<td><code>spark.yarn.access.hadoopFileSystems</code></td>
280280
<td>(none)</td>
281281
<td>
282-
A comma-separated list of secure HDFS namenodes your Spark application is going to access. For
283-
example, <code>spark.yarn.access.namenodes=hdfs://nn1.com:8032,hdfs://nn2.com:8032,
284-
webhdfs://nn3.com:50070</code>. The Spark application must have access to the namenodes listed
282+
A comma-separated list of secure Hadoop filesystems your Spark application is going to access. For
283+
example, <code>spark.yarn.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032,
284+
webhdfs://nn3.com:50070</code>. The Spark application must have access to the filesystems listed
285285
and Kerberos must be properly configured to be able to access them (either in the same realm
286-
or in a trusted realm). Spark acquires security tokens for each of the namenodes so that
287-
the Spark application can access those remote HDFS clusters.
286+
or in a trusted realm). Spark acquires security tokens for each of the filesystems so that
287+
the Spark application can access those remote Hadoop filesystems. <code>spark.yarn.access.namenodes</code>
288+
is deprecated, please use this instead.
288289
</td>
289290
</tr>
290291
<tr>
@@ -496,10 +497,10 @@ includes a URI of the metadata store in `"hive.metastore.uris`, and
496497

497498
If an application needs to interact with other secure Hadoop filesystems, then
498499
the tokens needed to access these clusters must be explicitly requested at
499-
launch time. This is done by listing them in the `spark.yarn.access.namenodes` property.
500+
launch time. This is done by listing them in the `spark.yarn.access.hadoopFileSystems` property.
500501

501502
```
502-
spark.yarn.access.namenodes hdfs://ireland.example.org:8020/,webhdfs://frankfurt.example.org:50070/
503+
spark.yarn.access.hadoopFileSystems hdfs://ireland.example.org:8020/,webhdfs://frankfurt.example.org:50070/
503504
```
504505

505506
Spark supports integrating with other security-aware services through Java Services mechanism (see
@@ -574,7 +575,7 @@ spark.yarn.security.credentials.hive.enabled false
574575
spark.yarn.security.credentials.hbase.enabled false
575576
```
576577

577-
The configuration option `spark.yarn.access.namenodes` must be unset.
578+
The configuration option `spark.yarn.access.hadoopFileSystems` must be unset.
578579

579580
## Troubleshooting Kerberos
580581

resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -248,6 +248,11 @@ package object config {
248248
.toSequence
249249
.createWithDefault(Nil)
250250

251+
private[spark] val FILESYSTEMS_TO_ACCESS = ConfigBuilder("spark.yarn.access.hadoopFileSystems")
252+
.doc("Extra Hadoop filesystem URLs for which to request delegation tokens. The filesystem " +
253+
"that hosts fs.defaultFS does not need to be listed here.")
254+
.fallbackConf(NAMENODES_TO_ACCESS)
255+
251256
/* Rolled log aggregation configuration. */
252257

253258
private[spark] val ROLLED_LOG_INCLUDE_PATTERN =

resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,7 @@ private[security] class HadoopFSCredentialProvider
4747
// NameNode to access, used to get tokens from different FileSystems
4848
val tmpCreds = new Credentials()
4949
val tokenRenewer = getTokenRenewer(hadoopConf)
50-
nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
50+
hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst =>
5151
val dstFs = dst.getFileSystem(hadoopConf)
5252
logInfo("getting token for: " + dst)
5353
dstFs.addDelegationTokens(tokenRenewer, tmpCreds)
@@ -80,7 +80,7 @@ private[security] class HadoopFSCredentialProvider
8080
// user as renewer.
8181
sparkConf.get(PRINCIPAL).flatMap { renewer =>
8282
val creds = new Credentials()
83-
nnsToAccess(hadoopConf, sparkConf).foreach { dst =>
83+
hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst =>
8484
val dstFs = dst.getFileSystem(hadoopConf)
8585
dstFs.addDelegationTokens(renewer, creds)
8686
}
@@ -112,8 +112,8 @@ private[security] class HadoopFSCredentialProvider
112112
delegTokenRenewer
113113
}
114114

115-
private def nnsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = {
116-
sparkConf.get(NAMENODES_TO_ACCESS).map(new Path(_)).toSet +
115+
private def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = {
116+
sparkConf.get(FILESYSTEMS_TO_ACCESS).map(new Path(_)).toSet +
117117
sparkConf.get(STAGING_DIR).map(new Path(_))
118118
.getOrElse(FileSystem.get(hadoopConf).getHomeDirectory)
119119
}

0 commit comments

Comments
 (0)