XComp commented on a change in pull request #15131:
URL: https://github.com/apache/flink/pull/15131#discussion_r623016843
##########
File path:
flink-core/src/main/java/org/apache/flink/configuration/SecurityOptions.java
##########
@@ -108,6 +108,20 @@
+ " (for example, `Client,KafkaClient` to
use the credentials for ZooKeeper authentication and for"
+ " Kafka authentication)");
+ @Documentation.Section(Documentation.Sections.SECURITY_AUTH_KERBEROS)
+ public static final ConfigOption<Boolean> KERBEROS_FETCH_DELEGATION_TOKEN =
+ key("security.kerberos.fetch.delegation-token")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription(
+ "Indicates whether to fetch the delegation tokens
for external services the Flink job needs to contact. "
+ + "Only HDFS and HBase are supported. It
is used in Yarn deployments. "
+ + "If true, Flink will fetch HDFS and
HBase delegation tokens and inject them into Yarn AM containers. "
+ + "If false, Flink will assume that the
delegation tokens are managed outside of Flink. "
+ + "As a consequence, it will not fetch
delegation tokens for HDFS and HBase. "
+ + "You may need to disable this option, if
you rely on submission mechanisms, e.g. Apache Oozie, "
+ + "to handle delegation tokens. ");
Review comment:
```suggestion
+ "to handle delegation tokens.");
```
a minor thing
##########
File path: flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
##########
@@ -197,21 +196,28 @@ private static LocalResource registerLocalResource(
}
public static void setTokensFor(
- ContainerLaunchContext amContainer, List<Path> paths,
Configuration conf)
+ ContainerLaunchContext amContainer,
+ List<Path> paths,
+ Configuration conf,
+ boolean obtainingDelegationTokens)
throws IOException {
Credentials credentials = new Credentials();
- // for HDFS
- TokenCache.obtainTokensForNamenodes(credentials, paths.toArray(new
Path[0]), conf);
- // for HBase
- obtainTokenForHBase(credentials, conf);
+
+ if (obtainingDelegationTokens) {
+ LOG.info("Obtaining delegation tokens for HDFS and HBase.");
+ // for HDFS
+ TokenCache.obtainTokensForNamenodes(credentials, paths.toArray(new
Path[0]), conf);
+ // for HBase
+ obtainTokenForHBase(credentials, conf);
+ }
+
// for user
UserGroupInformation currUsr = UserGroupInformation.getCurrentUser();
Collection<Token<? extends TokenIdentifier>> usrTok =
currUsr.getTokens();
for (Token<? extends TokenIdentifier> token : usrTok) {
- final Text id = new Text(token.getIdentifier());
- LOG.info("Adding user token " + id + " with " + token);
- credentials.addToken(id, token);
+ LOG.info("Adding user token " + token.getService() + " with " +
token);
Review comment:
Iterating over it once more, I realized that
[HadoopModule](https://github.com/apache/flink/blob/c6997c97c575d334679915c328792b8a3067cfb5/flink-runtime/src/main/java/org/apache/flink/runtime/security/modules/HadoopModule.java#L101)
uses the identifier as well. In contrast, the [HBase delegation
token](https://github.com/apache/flink/blob/e024401a27cc596f062b4686c4c38bbab25278b2/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java#L302)
always used `getService`. Do we have to align that? Moreover, I'm wondering
whether we should make this a dedicated ticket to align it as it's not
necessary for this feature. That would make this issue more transparent. Do you
have any objections against that, @zuston ?
##########
File path: flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java
##########
@@ -197,21 +196,28 @@ private static LocalResource registerLocalResource(
}
public static void setTokensFor(
- ContainerLaunchContext amContainer, List<Path> paths,
Configuration conf)
+ ContainerLaunchContext amContainer,
+ List<Path> paths,
+ Configuration conf,
+ boolean obtainingDelegationTokens)
throws IOException {
Credentials credentials = new Credentials();
- // for HDFS
- TokenCache.obtainTokensForNamenodes(credentials, paths.toArray(new
Path[0]), conf);
- // for HBase
- obtainTokenForHBase(credentials, conf);
+
+ if (obtainingDelegationTokens) {
+ LOG.info("Obtaining delegation tokens for HDFS and HBase.");
+ // for HDFS
+ TokenCache.obtainTokensForNamenodes(credentials, paths.toArray(new
Path[0]), conf);
+ // for HBase
+ obtainTokenForHBase(credentials, conf);
+ }
Review comment:
Have we discussed adding an `else` branch containing an info log message
pointing out that delegation token retrieval for HDFS and HBase is disabled?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]