zuston commented on a change in pull request #14841: URL: https://github.com/apache/flink/pull/14841#discussion_r633250018
########## File path: flink-yarn/src/main/java/org/apache/flink/yarn/security/HadoopDelegationTokenManager.java ########## @@ -0,0 +1,96 @@ +/* + * 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.runtime.security.delegationtokens.HadoopDelegationTokenConfiguration; +import org.apache.flink.runtime.security.delegationtokens.HadoopDelegationTokenProvider; + +import org.apache.hadoop.security.Credentials; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.ServiceLoader; + +/** + * HadoopDelegationTokenManager is responsible for managing delegation tokens. It can be used to + * obtain delegation tokens by calling `obtainDelegationTokens` method. + */ +public class HadoopDelegationTokenManager { + private static final Logger LOG = LoggerFactory.getLogger(HadoopDelegationTokenManager.class); + + private final HadoopDelegationTokenConfiguration hadoopDelegationTokenConf; + private final List<HadoopDelegationTokenProvider> delegationTokenProviders; + + public HadoopDelegationTokenManager( + HadoopDelegationTokenConfiguration hadoopDelegationTokenConf) { + this.hadoopDelegationTokenConf = hadoopDelegationTokenConf; + delegationTokenProviders = loadProviders(); + } + + /** + * Obtain delegation tokens using HadoopDelegationProviders, and store them in the give + * credentials. + * + * @param credentials Credentials object where to store the delegation tokens. + */ + public void obtainDelegationTokens(Credentials credentials) { + delegationTokenProviders.forEach( + provider -> { + if (provider.delegationTokensRequired()) { + provider.obtainDelegationTokens(credentials); + } else { + LOG.info( + "Service {} does not need to require a token,", + provider.serviceName()); + } + }); + } + + private List<HadoopDelegationTokenProvider> loadProviders() { + ServiceLoader<HadoopDelegationTokenProvider> serviceLoader = + ServiceLoader.load(HadoopDelegationTokenProvider.class); + + List<HadoopDelegationTokenProvider> providers = new ArrayList<>(); + + Iterator<HadoopDelegationTokenProvider> providerIterator = serviceLoader.iterator(); + providerIterator.forEachRemaining( + provider -> { + try { + provider.init(hadoopDelegationTokenConf); Review comment: Add logs that indicates which is hadoop delegation token provider loaded? ########## File path: flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java ########## @@ -197,13 +196,19 @@ private static LocalResource registerLocalResource( } public static void setTokensFor( - ContainerLaunchContext amContainer, List<Path> paths, Configuration conf) + org.apache.flink.configuration.Configuration flinkConf, + ContainerLaunchContext amContainer, + Configuration hadoopConf) throws IOException { Credentials credentials = new Credentials(); - // for HDFS - TokenCache.obtainTokensForNamenodes(credentials, paths.toArray(new Path[0]), conf); - // for HBase - obtainTokenForHBase(credentials, conf); + + // obtain tokens from HadoopDelegationTokenProviders Review comment: Need to be compatible with https://issues.apache.org/jira/browse/FLINK-21700# ########## 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); + fs.addDelegationTokens(renewer, credentials); + } catch (IOException e) { + LOG.warn("Failed to get token for {}.", fs); Review comment: Why throw exception when obtaining DT failed? ########## 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, it will be shown like as follows: `Getting FS token for: org.apache.hadoop.fs.LocalFileSystem@1a4927d6 with renewer xxxxx` ########## 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]
