Skip to content

Commit

Permalink
[FLINK-18205][security] Mitigate the use of reflection in HadoopModule
Browse files Browse the repository at this point in the history
  • Loading branch information
KarmaGYZ authored and zentol committed Jun 19, 2020
1 parent bff6038 commit 9bbe503
Showing 1 changed file with 15 additions and 34 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -83,42 +83,23 @@ public void install() throws SecurityInstallException {
// supplement with any available tokens
String fileLocation = System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
if (fileLocation != null) {
// Use reflection API since the API semantics are not available in Hadoop1 profile. Below APIs are
// used in the context of reading the stored tokens from UGI.
// Credentials cred = Credentials.readTokenStorageFile(new File(fileLocation), config.hadoopConf);
// loginUser.addCredentials(cred);
try {
Method readTokenStorageFileMethod = Credentials.class.getMethod("readTokenStorageFile",
File.class, org.apache.hadoop.conf.Configuration.class);
Credentials credentialsFromTokenStorageFile =
(Credentials) readTokenStorageFileMethod.invoke(
null,
new File(fileLocation),
hadoopConfiguration);

// if UGI uses Kerberos keytabs for login, do not load HDFS delegation token since
// the UGI would prefer the delegation token instead, which eventually expires
// and does not fallback to using Kerberos tickets
Method getAllTokensMethod = Credentials.class.getMethod("getAllTokens");
Credentials credentialsToBeAdded = new Credentials();
final Text hdfsDelegationTokenKind = new Text("HDFS_DELEGATION_TOKEN");
Collection<Token<? extends TokenIdentifier>> usrTok = (Collection<Token<? extends TokenIdentifier>>) getAllTokensMethod.invoke(credentialsFromTokenStorageFile);
//If UGI use keytab for login, do not load HDFS delegation token.
for (Token<? extends TokenIdentifier> token : usrTok) {
if (!token.getKind().equals(hdfsDelegationTokenKind)) {
final Text id = new Text(token.getIdentifier());
credentialsToBeAdded.addToken(id, token);
}
Credentials credentialsFromTokenStorageFile = Credentials.readTokenStorageFile(new File(fileLocation), hadoopConfiguration);

// if UGI uses Kerberos keytabs for login, do not load HDFS delegation token since
// the UGI would prefer the delegation token instead, which eventually expires
// and does not fallback to using Kerberos tickets
Credentials credentialsToBeAdded = new Credentials();
final Text hdfsDelegationTokenKind = new Text("HDFS_DELEGATION_TOKEN");
Collection<Token<? extends TokenIdentifier>> usrTok = credentialsFromTokenStorageFile.getAllTokens();
//If UGI use keytab for login, do not load HDFS delegation token.
for (Token<? extends TokenIdentifier> token : usrTok) {
if (!token.getKind().equals(hdfsDelegationTokenKind)) {
final Text id = new Text(token.getIdentifier());
credentialsToBeAdded.addToken(id, token);
}

Method addCredentialsMethod = UserGroupInformation.class.getMethod("addCredentials",
Credentials.class);
addCredentialsMethod.invoke(loginUser, credentialsToBeAdded);
} catch (NoSuchMethodException e) {
LOG.warn("Could not find method implementations in the shaded jar.", e);
} catch (InvocationTargetException e) {
throw e.getTargetException();
}

loginUser.addCredentials(credentialsToBeAdded);
}
} else {
// login with current user credentials (e.g. ticket cache, OS login)
Expand Down

0 comments on commit 9bbe503

Please sign in to comment.