HADOOP-15889. Add hadoop.token configuration parameter to load tokens. Contributed by Íñigo Goiri
This commit is contained in:
parent
05e4ddeee6
commit
0d61facd37
@ -630,6 +630,8 @@ public class CommonConfigurationKeysPublic {
|
|||||||
*/
|
*/
|
||||||
public static final String HADOOP_TOKEN_FILES =
|
public static final String HADOOP_TOKEN_FILES =
|
||||||
"hadoop.token.files";
|
"hadoop.token.files";
|
||||||
|
public static final String HADOOP_TOKENS =
|
||||||
|
"hadoop.tokens";
|
||||||
public static final String HADOOP_HTTP_AUTHENTICATION_TYPE =
|
public static final String HADOOP_HTTP_AUTHENTICATION_TYPE =
|
||||||
"hadoop.http.authentication.type";
|
"hadoop.http.authentication.type";
|
||||||
|
|
||||||
|
@ -23,13 +23,14 @@
|
|||||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED;
|
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED;
|
||||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED_DEFAULT;
|
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_KEYTAB_LOGIN_AUTORENEWAL_ENABLED_DEFAULT;
|
||||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
|
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
|
||||||
|
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKENS;
|
||||||
import static org.apache.hadoop.security.UGIExceptionMessages.*;
|
import static org.apache.hadoop.security.UGIExceptionMessages.*;
|
||||||
import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
|
import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
|
||||||
|
import static org.apache.hadoop.util.StringUtils.getTrimmedStringCollection;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileNotFoundException;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.lang.reflect.UndeclaredThrowableException;
|
import java.lang.reflect.UndeclaredThrowableException;
|
||||||
import java.security.AccessControlContext;
|
import java.security.AccessControlContext;
|
||||||
@ -42,10 +43,10 @@
|
|||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Date;
|
|
||||||
import java.util.EnumMap;
|
import java.util.EnumMap;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
import java.util.LinkedHashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
@ -88,7 +89,6 @@
|
|||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||||
import org.apache.hadoop.util.Shell;
|
import org.apache.hadoop.util.Shell;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
|
||||||
import org.apache.hadoop.util.Time;
|
import org.apache.hadoop.util.Time;
|
||||||
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
@ -298,7 +298,9 @@ public static void reattachMetrics() {
|
|||||||
|
|
||||||
/**Environment variable pointing to the token cache file*/
|
/**Environment variable pointing to the token cache file*/
|
||||||
public static final String HADOOP_TOKEN_FILE_LOCATION =
|
public static final String HADOOP_TOKEN_FILE_LOCATION =
|
||||||
"HADOOP_TOKEN_FILE_LOCATION";
|
"HADOOP_TOKEN_FILE_LOCATION";
|
||||||
|
/** Environment variable pointing to the base64 tokens. */
|
||||||
|
public static final String HADOOP_TOKEN = "HADOOP_TOKEN";
|
||||||
|
|
||||||
public static boolean isInitialized() {
|
public static boolean isInitialized() {
|
||||||
return conf != null;
|
return conf != null;
|
||||||
@ -752,45 +754,58 @@ UserGroupInformation createLoginUser(Subject subject) throws IOException {
|
|||||||
}
|
}
|
||||||
loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser);
|
loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser);
|
||||||
|
|
||||||
String tokenFileLocation = System.getProperty(HADOOP_TOKEN_FILES);
|
// Load tokens from files
|
||||||
if (tokenFileLocation == null) {
|
final Collection<String> tokenFileLocations = new LinkedHashSet<>();
|
||||||
tokenFileLocation = conf.get(HADOOP_TOKEN_FILES);
|
tokenFileLocations.addAll(getTrimmedStringCollection(
|
||||||
}
|
System.getProperty(HADOOP_TOKEN_FILES)));
|
||||||
if (tokenFileLocation != null) {
|
tokenFileLocations.addAll(getTrimmedStringCollection(
|
||||||
for (String tokenFileName:
|
conf.get(HADOOP_TOKEN_FILES)));
|
||||||
StringUtils.getTrimmedStrings(tokenFileLocation)) {
|
tokenFileLocations.addAll(getTrimmedStringCollection(
|
||||||
if (tokenFileName.length() > 0) {
|
System.getenv(HADOOP_TOKEN_FILE_LOCATION)));
|
||||||
File tokenFile = new File(tokenFileName);
|
for (String tokenFileLocation : tokenFileLocations) {
|
||||||
if (tokenFile.exists() && tokenFile.isFile()) {
|
if (tokenFileLocation != null && tokenFileLocation.length() > 0) {
|
||||||
Credentials cred = Credentials.readTokenStorageFile(
|
File tokenFile = new File(tokenFileLocation);
|
||||||
tokenFile, conf);
|
LOG.debug("Reading credentials from location {}",
|
||||||
loginUser.addCredentials(cred);
|
tokenFile.getCanonicalPath());
|
||||||
} else {
|
if (tokenFile.exists() && tokenFile.isFile()) {
|
||||||
LOG.info("tokenFile("+tokenFileName+") does not exist");
|
Credentials cred = Credentials.readTokenStorageFile(
|
||||||
}
|
tokenFile, conf);
|
||||||
|
LOG.debug("Loaded {} tokens from {}", cred.numberOfTokens(),
|
||||||
|
tokenFile.getCanonicalPath());
|
||||||
|
loginUser.addCredentials(cred);
|
||||||
|
} else {
|
||||||
|
LOG.info("Token file {} does not exist",
|
||||||
|
tokenFile.getCanonicalPath());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
String fileLocation = System.getenv(HADOOP_TOKEN_FILE_LOCATION);
|
// Load tokens from base64 encoding
|
||||||
if (fileLocation != null) {
|
final Collection<String> tokensBase64 = new LinkedHashSet<>();
|
||||||
// Load the token storage file and put all of the tokens into the
|
tokensBase64.addAll(getTrimmedStringCollection(
|
||||||
// user. Don't use the FileSystem API for reading since it has a lock
|
System.getProperty(HADOOP_TOKENS)));
|
||||||
// cycle (HADOOP-9212).
|
tokensBase64.addAll(getTrimmedStringCollection(
|
||||||
File source = new File(fileLocation);
|
conf.get(HADOOP_TOKENS)));
|
||||||
LOG.debug("Reading credentials from location set in {}: {}",
|
tokensBase64.addAll(getTrimmedStringCollection(
|
||||||
HADOOP_TOKEN_FILE_LOCATION,
|
System.getenv(HADOOP_TOKEN)));
|
||||||
source.getCanonicalPath());
|
int numTokenBase64 = 0;
|
||||||
if (!source.isFile()) {
|
for (String tokenBase64 : tokensBase64) {
|
||||||
throw new FileNotFoundException("Source file "
|
if (tokenBase64 != null && tokenBase64.length() > 0) {
|
||||||
+ source.getCanonicalPath() + " from "
|
try {
|
||||||
+ HADOOP_TOKEN_FILE_LOCATION
|
Token<TokenIdentifier> token = new Token<>();
|
||||||
+ " not found");
|
token.decodeFromUrlString(tokenBase64);
|
||||||
|
Credentials cred = new Credentials();
|
||||||
|
cred.addToken(token.getService(), token);
|
||||||
|
loginUser.addCredentials(cred);
|
||||||
|
numTokenBase64++;
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.error("Cannot add token {}: {}",
|
||||||
|
tokenBase64, ioe.getMessage());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
Credentials cred = Credentials.readTokenStorageFile(
|
}
|
||||||
source, conf);
|
if (numTokenBase64 > 0) {
|
||||||
LOG.debug("Loaded {} tokens", cred.numberOfTokens());
|
LOG.debug("Loaded {} base64 tokens", numTokenBase64);
|
||||||
loginUser.addCredentials(cred);
|
|
||||||
}
|
}
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
LOG.debug("failure to load login credentials", ioe);
|
LOG.debug("failure to load login credentials", ioe);
|
||||||
|
@ -1056,6 +1056,7 @@ public void testExternalTokenFiles() throws Exception {
|
|||||||
Collection<Token<?>> credsugiTokens = tokenUgi.getTokens();
|
Collection<Token<?>> credsugiTokens = tokenUgi.getTokens();
|
||||||
assertTrue(credsugiTokens.contains(token1));
|
assertTrue(credsugiTokens.contains(token1));
|
||||||
assertTrue(credsugiTokens.contains(token2));
|
assertTrue(credsugiTokens.contains(token2));
|
||||||
|
System.clearProperty("hadoop.token.files");
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -1270,4 +1271,96 @@ ugi.new TicketCacheRenewalRunnable(tgt,
|
|||||||
// isDestroyed should be called at least once
|
// isDestroyed should be called at least once
|
||||||
Mockito.verify(tgt, atLeastOnce()).isDestroyed();
|
Mockito.verify(tgt, atLeastOnce()).isDestroyed();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportTokensFromConfig() throws IOException {
|
||||||
|
Configuration config = new Configuration();
|
||||||
|
|
||||||
|
// Add a base64 token
|
||||||
|
String service0 = "testTokenImportService0";
|
||||||
|
byte[] identity = "identityImportConfig".getBytes();
|
||||||
|
byte[] password = "passwordImportConfig".getBytes();
|
||||||
|
Token<TokenIdentifier> expectedToken0 = new Token<>(identity, password,
|
||||||
|
new Text("testTokenKind0"), new Text(service0));
|
||||||
|
String tokenBase64 = expectedToken0.encodeToUrlString();
|
||||||
|
config.set(CommonConfigurationKeysPublic.HADOOP_TOKENS,
|
||||||
|
tokenBase64 + ",badtoken");
|
||||||
|
|
||||||
|
// Add a token from a file
|
||||||
|
String service1 = "testTokenImportService1";
|
||||||
|
Credentials cred0 = new Credentials();
|
||||||
|
Token<TokenIdentifier> expectedToken1 = expectedToken0.copyToken();
|
||||||
|
expectedToken1.setKind(new Text("testTokenKind1"));
|
||||||
|
expectedToken1.setService(new Text(service1));
|
||||||
|
cred0.addToken(expectedToken1.getService(), expectedToken1);
|
||||||
|
Path workDir = new Path(
|
||||||
|
GenericTestUtils.getRandomizedTestDir().getAbsolutePath());
|
||||||
|
Path tokenPath1 = new Path(workDir, "dt.token");
|
||||||
|
cred0.writeTokenStorageFile(tokenPath1, config);
|
||||||
|
config.set(CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES,
|
||||||
|
tokenPath1 + "," + new Path(workDir, "badfile"));
|
||||||
|
|
||||||
|
UserGroupInformation.reset();
|
||||||
|
UserGroupInformation.setConfiguration(config);
|
||||||
|
|
||||||
|
// Check if the tokens were loaded
|
||||||
|
UserGroupInformation ugi = UserGroupInformation.getLoginUser();
|
||||||
|
Credentials outCred = ugi.getCredentials();
|
||||||
|
assertEquals("Tokens: " + outCred.getAllTokens(),
|
||||||
|
2, outCred.getAllTokens().size());
|
||||||
|
boolean found0 = false;
|
||||||
|
boolean found1 = false;
|
||||||
|
for (Token<? extends TokenIdentifier> token : outCred.getAllTokens()) {
|
||||||
|
assertArrayEquals(identity, token.getIdentifier());
|
||||||
|
if (token.getService().toString().equals(service0)) {
|
||||||
|
assertEquals(expectedToken0.encodeToUrlString(),
|
||||||
|
token.encodeToUrlString());
|
||||||
|
found0 = true;
|
||||||
|
}
|
||||||
|
if (token.getService().toString().equals(service1)) {
|
||||||
|
found1 = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
assertTrue("Expected token testTokenService0 not found: " + outCred,
|
||||||
|
found0);
|
||||||
|
assertTrue("Expected token testTokenService1 not found: " + outCred,
|
||||||
|
found1);
|
||||||
|
|
||||||
|
// Try to add the same token through configuration and file
|
||||||
|
Credentials cred1 = new Credentials();
|
||||||
|
cred1.addToken(expectedToken0.getService(), expectedToken0);
|
||||||
|
cred1.writeTokenStorageFile(tokenPath1, config);
|
||||||
|
|
||||||
|
UserGroupInformation.reset();
|
||||||
|
UserGroupInformation.setConfiguration(config);
|
||||||
|
|
||||||
|
UserGroupInformation ugi1 = UserGroupInformation.getLoginUser();
|
||||||
|
Credentials outCred1 = ugi1.getCredentials();
|
||||||
|
assertEquals("Tokens: " + outCred1.getAllTokens(),
|
||||||
|
1, outCred1.getAllTokens().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImportTokensFromProperty() throws IOException {
|
||||||
|
// Add a base64 token
|
||||||
|
Text service = new Text("testTokenProperty");
|
||||||
|
byte[] identity = "identityImportProperty".getBytes();
|
||||||
|
byte[] password = "passwordImportProperty".getBytes();
|
||||||
|
Token<TokenIdentifier> expectedToken0 = new Token<>(identity, password,
|
||||||
|
new Text("testTokenKind0"), service);
|
||||||
|
String tokenBase64 = expectedToken0.encodeToUrlString();
|
||||||
|
System.setProperty(CommonConfigurationKeysPublic.HADOOP_TOKENS,
|
||||||
|
tokenBase64);
|
||||||
|
|
||||||
|
// Check if the tokens were loaded
|
||||||
|
UserGroupInformation.reset();
|
||||||
|
UserGroupInformation ugi = UserGroupInformation.getLoginUser();
|
||||||
|
Credentials creds = ugi.getCredentials();
|
||||||
|
assertEquals("Tokens: " + creds.getAllTokens(),
|
||||||
|
1, creds.getAllTokens().size());
|
||||||
|
assertArrayEquals(creds.getToken(service).getIdentifier(), identity);
|
||||||
|
|
||||||
|
// Cleanup
|
||||||
|
System.clearProperty(CommonConfigurationKeysPublic.HADOOP_TOKENS);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user