aboutsummaryrefslogtreecommitdiff
path: root/resource-managers
diff options
context:
space:
mode:
Diffstat (limited to 'resource-managers')
-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
2 files changed, 9 insertions, 4 deletions
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)
}