Skip to content

Commit 1c87238

Browse files
committed
Review fixes:
* Simplified hadoopFSsToAccess * Moved doc to generic area
1 parent 32c5d5d commit 1c87238

File tree

3 files changed

+21
-22
lines changed

3 files changed

+21
-22
lines changed

core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -143,17 +143,16 @@ private[deploy] object HadoopFSDelegationTokenProvider {
143143
val defaultFS = FileSystem.get(hadoopConf)
144144
val master = sparkConf.get("spark.master", null)
145145
val stagingFS = if (master != null && master.contains("yarn")) {
146-
sparkConf.get(STAGING_DIR)
147-
.map(new Path(_).getFileSystem(hadoopConf))
148-
.getOrElse(defaultFS)
146+
sparkConf.get(STAGING_DIR).map(new Path(_).getFileSystem(hadoopConf))
149147
} else {
150-
defaultFS
148+
None
151149
}
152150

153151
// Add the list of available namenodes for all namespaces in HDFS federation.
154152
// If ViewFS is enabled, this is skipped as ViewFS already handles delegation tokens for its
155153
// namespaces.
156-
val hadoopFilesystems = if (!filesystemsToAccess.isEmpty || stagingFS.getScheme == "viewfs") {
154+
val hadoopFilesystems = if (!filesystemsToAccess.isEmpty || defaultFS.getScheme == "viewfs" ||
155+
(stagingFS.isDefined && stagingFS.get.getScheme == "viewfs")) {
157156
filesystemsToAccess.map(new Path(_).getFileSystem(hadoopConf)).toSet
158157
} else {
159158
val nameservices = hadoopConf.getTrimmedStrings("dfs.nameservices")
@@ -172,6 +171,6 @@ private[deploy] object HadoopFSDelegationTokenProvider {
172171
(filesystemsWithoutHA ++ filesystemsWithHA).toSet
173172
}
174173

175-
hadoopFilesystems + stagingFS + defaultFS
174+
hadoopFilesystems ++ stagingFS + defaultFS
176175
}
177176
}

docs/running-on-yarn.md

Lines changed: 0 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -490,10 +490,6 @@ for:
490490
filesystem if `spark.yarn.stagingDir` is not set);
491491
- if Hadoop federation is enabled, all the federated filesystems in the configuration.
492492

493-
If an application needs to interact with other secure Hadoop filesystems, their URIs need to be
494-
explicitly provided to Spark at launch time. This is done by listing them in the
495-
`spark.kerberos.access.hadoopFileSystems` property, described in the configuration section below.
496-
497493
The YARN integration also supports custom delegation token providers using the Java Services
498494
mechanism (see `java.util.ServiceLoader`). Implementations of
499495
`org.apache.spark.deploy.yarn.security.ServiceCredentialProvider` can be made available to Spark
@@ -527,18 +523,6 @@ providers can be disabled individually by setting `spark.security.credentials.{s
527523
<br /> (Works also with the "local" master.)
528524
</td>
529525
</tr>
530-
<tr>
531-
<td><code>spark.kerberos.access.hadoopFileSystems</code></td>
532-
<td>(none)</td>
533-
<td>
534-
A comma-separated list of secure Hadoop filesystems your Spark application is going to access. For
535-
example, <code>spark.kerberos.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032,
536-
webhdfs://nn3.com:50070</code>. The Spark application must have access to the filesystems listed
537-
and Kerberos must be properly configured to be able to access them (either in the same realm
538-
or in a trusted realm). Spark acquires security tokens for each of the filesystems so that
539-
the Spark application can access those remote Hadoop filesystems.
540-
</td>
541-
</tr>
542526
<tr>
543527
<td><code>spark.yarn.kerberos.relogin.period</code></td>
544528
<td>1m</td>

docs/security.md

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -752,6 +752,10 @@ configuration has Kerberos authentication turned (`hbase.security.authentication
752752
Similarly, a Hive token will be obtained if Hive is in the classpath, and the configuration includes
753753
URIs for remote metastore services (`hive.metastore.uris` is not empty).
754754

755+
If an application needs to interact with other secure Hadoop filesystems, their URIs need to be
756+
explicitly provided to Spark at launch time. This is done by listing them in the
757+
`spark.kerberos.access.hadoopFileSystems` property, described in the configuration section below.
758+
755759
Delegation token support is currently only supported in YARN and Mesos modes. Consult the
756760
deployment-specific page for more information.
757761

@@ -769,6 +773,18 @@ The following options provides finer-grained control for this feature:
769773
application being run.
770774
</td>
771775
</tr>
776+
<tr>
777+
<td><code>spark.kerberos.access.hadoopFileSystems</code></td>
778+
<td>(none)</td>
779+
<td>
780+
A comma-separated list of secure Hadoop filesystems your Spark application is going to access. For
781+
example, <code>spark.kerberos.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032,
782+
webhdfs://nn3.com:50070</code>. The Spark application must have access to the filesystems listed
783+
and Kerberos must be properly configured to be able to access them (either in the same realm
784+
or in a trusted realm). Spark acquires security tokens for each of the filesystems so that
785+
the Spark application can access those remote Hadoop filesystems.
786+
</td>
787+
</tr>
772788
</table>
773789

774790
## Long-Running Applications

0 commit comments

Comments
 (0)