zuston commented on a change in pull request #14841:
URL: https://github.com/apache/flink/pull/14841#discussion_r633255170



##########
File path: 
flink-yarn/src/main/java/org/apache/flink/yarn/security/HadoopFSDelegationTokenProvider.java
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.yarn.security;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigUtils;
+import 
org.apache.flink.runtime.security.delegationtokens.HadoopDelegationTokenConfiguration;
+import 
org.apache.flink.runtime.security.delegationtokens.HadoopDelegationTokenProvider;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.function.FunctionUtils;
+import org.apache.flink.yarn.configuration.YarnConfigOptions;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.Master;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Delegation token provider implementation for Hadoop FileSystems. */
+public class HadoopFSDelegationTokenProvider implements 
HadoopDelegationTokenProvider {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(HadoopFSDelegationTokenProvider.class);
+
+    private HadoopDelegationTokenConfiguration hadoopDelegationTokenConf;
+
+    @Override
+    public String serviceName() {
+        return "hadoopfs";
+    }
+
+    @Override
+    public void init(final HadoopDelegationTokenConfiguration conf) {
+        this.hadoopDelegationTokenConf = conf;
+    }
+
+    @Override
+    public boolean delegationTokensRequired() {
+        return UserGroupInformation.isSecurityEnabled();
+    }
+
+    @Override
+    public Optional<Long> obtainDelegationTokens(final Credentials 
credentials) {
+        try {
+            Set<FileSystem> fileSystemsToAccess = getFileSystemsToAccess();
+
+            final String renewer = 
getTokenRenewer(hadoopDelegationTokenConf.getHadoopConf());
+            fileSystemsToAccess.forEach(
+                    fs -> {
+                        try {
+                            LOG.info("Getting FS token for: {} with renewer 
{}", fs, renewer);

Review comment:
       This log will makes users confused. Because it dont indicate which path 
is getting fs token. 
   
   Besides, I think it is inappropriate for the fs object to be used directly 
in the log msg, it will be shown like as follows:
   
   `Getting FS token for: org.apache.hadoop.fs.LocalFileSystem@1a4927d6 with 
renewer xxxxx`




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