XComp commented on a change in pull request #15131:
URL: https://github.com/apache/flink/pull/15131#discussion_r622892092



##########
File path: 
flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
##########
@@ -529,6 +529,16 @@ public void killCluster(ApplicationId applicationId) 
throws FlinkException {
                         "Hadoop security with Kerberos is enabled but the 
login user "
                                 + "does not have Kerberos credentials or 
delegation tokens!");
             }
+
+            boolean fetchToken =
+                    
flinkConfiguration.getBoolean(SecurityOptions.KERBEROS_FETCH_DELEGATION_TOKEN);
+            boolean yarnAccessFSEnabled =
+                    flinkConfiguration.get(YarnConfigOptions.YARN_ACCESS) != 
null;
+            if (!fetchToken && yarnAccessFSEnabled) {
+                throw new IllegalConfigurationException(
+                        "When security.kerberos.fetch.delegation-token is set, 
"
+                                + 
"yarn.security.kerberos.additionalFileSystems must be unset.");

Review comment:
       ```suggestion
                   throw new IllegalConfigurationException(
                           String.format(
                                   "When %s is disabled, %s must be disabled as 
well.",
                                   
SecurityOptions.KERBEROS_FETCH_DELEGATION_TOKEN.key(),
                                   YarnConfigOptions.YARN_ACCESS.key()));
   ```
   
   The message was wrong, wasn't it? Additionally, it's better to use the 
references to the parameters instead of plain text.

##########
File path: 
flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
##########
@@ -1081,13 +1081,22 @@ private ApplicationReport startAppMaster(
         if (UserGroupInformation.isSecurityEnabled()) {
             // set HDFS delegation tokens when security is enabled
             LOG.info("Adding delegation token to the AM container.");

Review comment:
       I verified it: The log message is still valid since `Utils.setTokensFor` 
also sets user-related tokens. We only disable HDFS and HBase when 
`SecurityOptions.KERBEROS_FETCH_DELEGATION_TOKEN` is disabled.

##########
File path: 
flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java
##########
@@ -1081,13 +1091,16 @@ private ApplicationReport startAppMaster(
         if (UserGroupInformation.isSecurityEnabled()) {
             // set HDFS delegation tokens when security is enabled
             LOG.info("Adding delegation token to the AM container.");
-            List<Path> yarnAccessList =
-                    ConfigUtils.decodeListFromConfig(
-                            configuration, YarnConfigOptions.YARN_ACCESS, 
Path::new);
-            Utils.setTokensFor(
-                    amContainer,
-                    ListUtils.union(yarnAccessList, 
fileUploader.getRemotePaths()),
-                    yarnConfiguration);
+            List<Path> pathsToObtainToken = null;

Review comment:
       ```suggestion
               List<Path> pathsToObtainToken = Collections.emptyList();
   ```
   I'm not that comfortable to pass `null` into `Utils.setTokensFor` as it's 
not a `@Nullable` parameter. It does not harm the execution because it's only 
used in `Utils.setTokensFor` if 
`SecurityOptions.KERBEROS_FETCH_DELEGATION_TOKEN` is enabled (and in that case 
we overwrite `pathsToObtainToken`). But still, I would vote for switching to an 
empty list instead.




-- 
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]


Reply via email to