HDFS-9525. hadoop utilities need to support provided delegation tokens (HeeSoo Kim via aw)

This commit is contained in:
Allen Wittenauer 2016-01-22 12:15:22 -08:00
parent 95363bcc7d
commit d22c4239a4
8 changed files with 92 additions and 14 deletions

View File

@ -311,6 +311,9 @@ public class CommonConfigurationKeysPublic {
/** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
public static final String HADOOP_SECURITY_DNS_NAMESERVER_KEY =
"hadoop.security.dns.nameserver";
/** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
public static final String HADOOP_TOKEN_FILES =
"hadoop.token.files";
/** See <a href="{@docRoot}/../core-default.html">core-default.xml</a> */
public static final String HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN =

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.security;
import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_METRICS_PERCENTILES_INTERVALS;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_KERBEROS_MIN_SECONDS_BEFORE_RELOGIN_DEFAULT;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES;
import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
import java.io.File;
@ -70,6 +71,7 @@ import org.apache.hadoop.security.authentication.util.KerberosUtil;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
@ -821,6 +823,26 @@ public class UserGroupInformation {
}
loginUser = proxyUser == null ? realUser : createProxyUser(proxyUser, realUser);
String tokenFileLocation = System.getProperty(HADOOP_TOKEN_FILES);
if (tokenFileLocation == null) {
tokenFileLocation = conf.get(HADOOP_TOKEN_FILES);
}
if (tokenFileLocation != null) {
for (String tokenFileName:
StringUtils.getTrimmedStrings(tokenFileLocation)) {
if (tokenFileName.length() > 0) {
File tokenFile = new File(tokenFileName);
if (tokenFile.exists() && tokenFile.isFile()) {
Credentials cred = Credentials.readTokenStorageFile(
tokenFile, conf);
loginUser.addCredentials(cred);
} else {
LOG.info("tokenFile("+tokenFileName+") does not exist");
}
}
}
}
String fileLocation = System.getenv(HADOOP_TOKEN_FILE_LOCATION);
if (fileLocation != null) {
// Load the token storage file and put all of the tokens into the

View File

@ -466,6 +466,12 @@ for ldap providers in the same way as above does.
<description>Maps kerberos principals to local user names</description>
</property>
<property>
<name>hadoop.token.files</name>
<value></value>
<description>List of token cache files that have delegation tokens for hadoop service</description>
</property>
<!-- i/o properties -->
<property>
<name>io.file.buffer.size</name>

View File

@ -18,6 +18,7 @@ package org.apache.hadoop.security;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
@ -35,6 +36,7 @@ import javax.security.auth.login.AppConfigurationEntry;
import javax.security.auth.login.LoginContext;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.lang.reflect.Method;
@ -848,7 +850,9 @@ public class TestUserGroupInformation {
*/
@Test
public void testPrivateTokenExclusion() throws Exception {
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
UserGroupInformation ugi =
UserGroupInformation.createUserForTesting(
"privateUser", new String[] { "PRIVATEUSERS" });
TestTokenIdentifier tokenId = new TestTokenIdentifier();
Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(
tokenId.getBytes(), "password".getBytes(),
@ -928,4 +932,46 @@ public class TestUserGroupInformation {
}
}
}
@Test
public void testExternalTokenFiles() throws Exception {
StringBuilder tokenFullPathnames = new StringBuilder();
String tokenFilenames = "token1,token2";
String tokenFiles[] = StringUtils.getTrimmedStrings(tokenFilenames);
final File testDir = new File("target",
TestUserGroupInformation.class.getName() + "-tmpDir").getAbsoluteFile();
String testDirPath = testDir.getAbsolutePath();
// create path for token files
for (String tokenFile: tokenFiles) {
if (tokenFullPathnames.length() > 0) {
tokenFullPathnames.append(",");
}
tokenFullPathnames.append(testDirPath).append("/").append(tokenFile);
}
// create new token and store it
TestTokenIdentifier tokenId = new TestTokenIdentifier();
Credentials cred1 = new Credentials();
Token<TestTokenIdentifier> token1 = new Token<TestTokenIdentifier>(
tokenId.getBytes(), "password".getBytes(),
tokenId.getKind(), new Text("token-service1"));
cred1.addToken(token1.getService(), token1);
cred1.writeTokenStorageFile(new Path(testDirPath, tokenFiles[0]), conf);
Credentials cred2 = new Credentials();
Token<TestTokenIdentifier> token2 = new Token<TestTokenIdentifier>(
tokenId.getBytes(), "password".getBytes(),
tokenId.getKind(), new Text("token-service2"));
cred2.addToken(token2.getService(), token2);
cred2.writeTokenStorageFile(new Path(testDirPath, tokenFiles[1]), conf);
// set property for token external token files
System.setProperty("hadoop.token.files", tokenFullPathnames.toString());
UserGroupInformation.setLoginUser(null);
UserGroupInformation tokenUgi = UserGroupInformation.getLoginUser();
Collection<Token<?>> credsugiTokens = tokenUgi.getTokens();
assertTrue(credsugiTokens.contains(token1));
assertTrue(credsugiTokens.contains(token2));
}
}

View File

@ -238,7 +238,7 @@ public class WebHdfsFileSystem extends FileSystem
// the first getAuthParams() for a non-token op will either get the
// internal token from the ugi or lazy fetch one
protected synchronized Token<?> getDelegationToken() throws IOException {
if (canRefreshDelegationToken && delegationToken == null) {
if (delegationToken == null) {
Token<?> token = tokenSelector.selectToken(
new Text(getCanonicalServiceName()), ugi.getTokens());
// ugi tokens are usually indicative of a task which can't
@ -248,11 +248,13 @@ public class WebHdfsFileSystem extends FileSystem
LOG.debug("Using UGI token: {}", token);
canRefreshDelegationToken = false;
} else {
token = getDelegationToken(null);
if (token != null) {
LOG.debug("Fetched new token: {}", token);
} else { // security is disabled
canRefreshDelegationToken = false;
if (canRefreshDelegationToken) {
token = getDelegationToken(null);
if (token != null) {
LOG.debug("Fetched new token: {}", token);
} else { // security is disabled
canRefreshDelegationToken = false;
}
}
}
setDelegationToken(token);

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hdfs.web.resources;
import org.apache.hadoop.security.UserGroupInformation;
/** Represents delegation token used for authentication. */
public class DelegationParam extends StringParam {
/** Parameter name. */
@ -33,8 +31,7 @@ public class DelegationParam extends StringParam {
* @param str a string representation of the parameter value.
*/
public DelegationParam(final String str) {
super(DOMAIN, UserGroupInformation.isSecurityEnabled()
&& str != null && !str.equals(DEFAULT)? str: null);
super(DOMAIN, str != null && !str.equals(DEFAULT)? str: null);
}
@Override

View File

@ -55,6 +55,9 @@ Trunk (Unreleased)
HDFS-9057. allow/disallow snapshots via webhdfs
(Bramma Reddy Battula via vinayakumarb)
HDFS-9525. hadoop utilities need to support provided delegation
tokens (HeeSoo Kim via aw)
IMPROVEMENTS
HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.

View File

@ -195,7 +195,7 @@ public class TestWebHdfsUrl {
checkQueryParams(
new String[]{
GetOpParam.Op.GETFILESTATUS.toQueryString(),
new UserParam(ugi.getShortUserName()).toString()
new DelegationParam(tokenString).toString()
},
fileStatusUrl);
}
@ -280,8 +280,7 @@ public class TestWebHdfsUrl {
checkQueryParams(
new String[]{
GetOpParam.Op.GETFILESTATUS.toQueryString(),
new UserParam(ugi.getRealUser().getShortUserName()).toString(),
new DoAsParam(ugi.getShortUserName()).toString()
new DelegationParam(tokenString).toString()
},
fileStatusUrl);
}