HADOOP-13251. Authenticate with Kerberos credentials when renewing KMS delegation token. Contributed by Xiao Chen.

This commit is contained in:
Andrew Wang 2016-06-27 18:20:56 -07:00
parent 9683eab0e1
commit 771f798edf
5 changed files with 153 additions and 95 deletions

View File

@ -339,7 +339,7 @@ public class UserGroupInformation {
@InterfaceAudience.Private @InterfaceAudience.Private
@VisibleForTesting @VisibleForTesting
static void reset() { public static void reset() {
authenticationMethod = null; authenticationMethod = null;
conf = null; conf = null;
groups = null; groups = null;

View File

@ -58,6 +58,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
private static final String HTTP_PUT = "PUT"; private static final String HTTP_PUT = "PUT";
public static final String OP_PARAM = "op"; public static final String OP_PARAM = "op";
private static final String OP_PARAM_EQUALS = OP_PARAM + "=";
public static final String DELEGATION_TOKEN_HEADER = public static final String DELEGATION_TOKEN_HEADER =
"X-Hadoop-Delegation-Token"; "X-Hadoop-Delegation-Token";
@ -285,13 +286,22 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
} }
url = new URL(sb.toString()); url = new URL(sb.toString());
AuthenticatedURL aUrl = new AuthenticatedURL(this, connConfigurator); AuthenticatedURL aUrl = new AuthenticatedURL(this, connConfigurator);
org.apache.hadoop.security.token.Token<AbstractDelegationTokenIdentifier>
dt = null;
if (token instanceof DelegationTokenAuthenticatedURL.Token
&& operation.requiresKerberosCredentials()) {
// Unset delegation token to trigger fall-back authentication.
dt = ((DelegationTokenAuthenticatedURL.Token) token).getDelegationToken();
((DelegationTokenAuthenticatedURL.Token) token).setDelegationToken(null);
}
try {
HttpURLConnection conn = aUrl.openConnection(url, token); HttpURLConnection conn = aUrl.openConnection(url, token);
conn.setRequestMethod(operation.getHttpMethod()); conn.setRequestMethod(operation.getHttpMethod());
HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK); HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
if (hasResponse) { if (hasResponse) {
String contentType = conn.getHeaderField(CONTENT_TYPE); String contentType = conn.getHeaderField(CONTENT_TYPE);
contentType = (contentType != null) ? StringUtils.toLowerCase(contentType) contentType =
: null; (contentType != null) ? StringUtils.toLowerCase(contentType) : null;
if (contentType != null && if (contentType != null &&
contentType.contains(APPLICATION_JSON_MIME)) { contentType.contains(APPLICATION_JSON_MIME)) {
try { try {
@ -308,6 +318,11 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
url.getAuthority(), operation)); url.getAuthority(), operation));
} }
} }
} finally {
if (dt != null) {
((DelegationTokenAuthenticatedURL.Token) token).setDelegationToken(dt);
}
}
return ret; return ret;
} }

View File

@ -30,6 +30,8 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
import org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager; import org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
@ -41,6 +43,8 @@ import com.google.common.annotations.VisibleForTesting;
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class DelegationTokenManager { public class DelegationTokenManager {
private static final Logger LOG =
LoggerFactory.getLogger(DelegationTokenManager.class);
public static final String ENABLE_ZK_KEY = "zk-dt-secret-manager.enable"; public static final String ENABLE_ZK_KEY = "zk-dt-secret-manager.enable";
@ -156,6 +160,7 @@ public class DelegationTokenManager {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public Token<? extends AbstractDelegationTokenIdentifier> createToken( public Token<? extends AbstractDelegationTokenIdentifier> createToken(
UserGroupInformation ugi, String renewer) { UserGroupInformation ugi, String renewer) {
LOG.debug("Creating token with ugi:{}, renewer:{}.", ugi, renewer);
renewer = (renewer == null) ? ugi.getShortUserName() : renewer; renewer = (renewer == null) ? ugi.getShortUserName() : renewer;
String user = ugi.getUserName(); String user = ugi.getUserName();
Text owner = new Text(user); Text owner = new Text(user);
@ -175,6 +180,7 @@ public class DelegationTokenManager {
public long renewToken( public long renewToken(
Token<? extends AbstractDelegationTokenIdentifier> token, String renewer) Token<? extends AbstractDelegationTokenIdentifier> token, String renewer)
throws IOException { throws IOException {
LOG.debug("Renewing token:{} with renewer:{}.", token, renewer);
return secretManager.renewToken(token, renewer); return secretManager.renewToken(token, renewer);
} }
@ -182,6 +188,7 @@ public class DelegationTokenManager {
public void cancelToken( public void cancelToken(
Token<? extends AbstractDelegationTokenIdentifier> token, Token<? extends AbstractDelegationTokenIdentifier> token,
String canceler) throws IOException { String canceler) throws IOException {
LOG.debug("Cancelling token:{} with canceler:{}.", token, canceler);
canceler = (canceler != null) ? canceler : canceler = (canceler != null) ? canceler :
verifyToken(token).getShortUserName(); verifyToken(token).getShortUserName();
secretManager.cancelToken(token, canceler); secretManager.cancelToken(token, canceler);

View File

@ -585,7 +585,7 @@ $H3 KMS Delegation Token Configuration
KMS supports delegation tokens to authenticate to the key providers from processes without Kerberos credentials. KMS supports delegation tokens to authenticate to the key providers from processes without Kerberos credentials.
KMS delegation token authentication extends the default Hadoop authentication. See [Hadoop Auth](../hadoop-auth/index.html) page for more details. KMS delegation token authentication extends the default Hadoop authentication. Same as Hadoop authentication, KMS delegation tokens must not be fetched or renewed using delegation token authentication. See [Hadoop Auth](../hadoop-auth/index.html) page for more details.
Additionally, KMS delegation token secret manager can be configured with the following properties: Additionally, KMS delegation token secret manager can be configured with the following properties:

View File

@ -35,13 +35,10 @@ import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
import org.apache.hadoop.security.authentication.client.Authenticator;
import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
import org.apache.hadoop.security.authorize.AuthorizationException; import org.apache.hadoop.security.authorize.AuthorizationException;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -73,11 +70,6 @@ import java.util.Properties;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
public class TestKMS { public class TestKMS {
private static final Logger LOG = LoggerFactory.getLogger(TestKMS.class); private static final Logger LOG = LoggerFactory.getLogger(TestKMS.class);
@ -262,6 +254,7 @@ public class TestKMS {
kdc = null; kdc = null;
} }
UserGroupInformation.setShouldRenewImmediatelyForTests(false); UserGroupInformation.setShouldRenewImmediatelyForTests(false);
UserGroupInformation.reset();
} }
private <T> T doAs(String user, final PrivilegedExceptionAction<T> action) private <T> T doAs(String user, final PrivilegedExceptionAction<T> action)
@ -1748,68 +1741,102 @@ public class TestKMS {
@Test @Test
public void testDelegationTokensOpsSimple() throws Exception { public void testDelegationTokensOpsSimple() throws Exception {
final Configuration conf = new Configuration(); final Configuration conf = new Configuration();
final Authenticator mock = mock(PseudoAuthenticator.class); testDelegationTokensOps(conf, false);
testDelegationTokensOps(conf, mock);
} }
@Test @Test
public void testDelegationTokensOpsKerberized() throws Exception { public void testDelegationTokensOpsKerberized() throws Exception {
final Configuration conf = new Configuration(); final Configuration conf = new Configuration();
conf.set("hadoop.security.authentication", "kerberos"); conf.set("hadoop.security.authentication", "kerberos");
final Authenticator mock = mock(KerberosAuthenticator.class); testDelegationTokensOps(conf, true);
testDelegationTokensOps(conf, mock);
} }
private void testDelegationTokensOps(Configuration conf, private void testDelegationTokensOps(Configuration conf,
final Authenticator mockAuthenticator) throws Exception { final boolean useKrb) throws Exception {
UserGroupInformation.setConfiguration(conf); UserGroupInformation.setConfiguration(conf);
File confDir = getTestDir(); File confDir = getTestDir();
conf = createBaseKMSConf(confDir); conf = createBaseKMSConf(confDir);
if (useKrb) {
conf.set("hadoop.kms.authentication.type", "kerberos");
conf.set("hadoop.kms.authentication.kerberos.keytab",
keytab.getAbsolutePath());
conf.set("hadoop.kms.authentication.kerberos.principal",
"HTTP/localhost");
conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
}
writeConf(confDir, conf); writeConf(confDir, conf);
doNothing().when(mockAuthenticator).authenticate(any(URL.class),
any(AuthenticatedURL.Token.class));
runServer(null, null, confDir, new KMSCallable<Void>() { runServer(null, null, confDir, new KMSCallable<Void>() {
@Override @Override
public Void call() throws Exception { public Void call() throws Exception {
Configuration conf = new Configuration(); final Configuration clientConf = new Configuration();
URI uri = createKMSUri(getKMSUrl()); final URI uri = createKMSUri(getKMSUrl());
KeyProvider kp = createProvider(uri, conf); clientConf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
createKMSUri(getKMSUrl()).toString()); createKMSUri(getKMSUrl()).toString());
doAs("client", new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
KeyProvider kp = createProvider(uri, clientConf);
// test delegation token retrieval // test delegation token retrieval
KeyProviderDelegationTokenExtension kpdte = KeyProviderDelegationTokenExtension kpdte =
KeyProviderDelegationTokenExtension. KeyProviderDelegationTokenExtension.
createKeyProviderDelegationTokenExtension(kp); createKeyProviderDelegationTokenExtension(kp);
Credentials credentials = new Credentials(); final Credentials credentials = new Credentials();
final Token<?>[] tokens = kpdte.addDelegationTokens( final Token<?>[] tokens =
UserGroupInformation.getCurrentUser().getUserName(), credentials); kpdte.addDelegationTokens("client1", credentials);
Assert.assertEquals(1, credentials.getAllTokens().size()); Assert.assertEquals(1, credentials.getAllTokens().size());
InetSocketAddress kmsAddr = new InetSocketAddress(getKMSUrl().getHost(), InetSocketAddress kmsAddr =
new InetSocketAddress(getKMSUrl().getHost(),
getKMSUrl().getPort()); getKMSUrl().getPort());
Assert.assertEquals(KMSClientProvider.TOKEN_KIND, Assert.assertEquals(KMSClientProvider.TOKEN_KIND,
credentials.getToken(SecurityUtil.buildTokenService(kmsAddr)). credentials.getToken(SecurityUtil.buildTokenService(kmsAddr)).
getKind()); getKind());
// After this point, we're supposed to use the delegation token to auth. // Test non-renewer user cannot renew.
doThrow(new IOException("Authenticator should not fall back"))
.when(mockAuthenticator).authenticate(any(URL.class),
any(AuthenticatedURL.Token.class));
// test delegation token renewal
boolean renewed = false;
for (Token<?> token : tokens) { for (Token<?> token : tokens) {
if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) { if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) {
LOG.info("Skipping token {}", token); LOG.info("Skipping token {}", token);
continue; continue;
} }
LOG.info("Got dt for " + uri + "; " + token); LOG.info("Got dt for " + uri + "; " + token);
long tokenLife = token.renew(conf); try {
token.renew(clientConf);
Assert.fail("client should not be allowed to renew token with"
+ "renewer=client1");
} catch (Exception e) {
GenericTestUtils.assertExceptionContains(
"tries to renew a token with renewer", e);
}
}
final UserGroupInformation otherUgi;
if (useKrb) {
UserGroupInformation
.loginUserFromKeytab("client1", keytab.getAbsolutePath());
otherUgi = UserGroupInformation.getLoginUser();
} else {
otherUgi = UserGroupInformation.createUserForTesting("client1",
new String[] {"other group"});
}
try {
// test delegation token renewal via renewer
otherUgi.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
boolean renewed = false;
for (Token<?> token : tokens) {
if (!(token.getKind()
.equals(KMSClientProvider.TOKEN_KIND))) {
LOG.info("Skipping token {}", token);
continue;
}
LOG.info("Got dt for " + uri + "; " + token);
long tokenLife = token.renew(clientConf);
LOG.info("Renewed token of kind {}, new lifetime:{}", LOG.info("Renewed token of kind {}, new lifetime:{}",
token.getKind(), tokenLife); token.getKind(), tokenLife);
Thread.sleep(100); Thread.sleep(100);
long newTokenLife = token.renew(conf); long newTokenLife = token.renew(clientConf);
LOG.info("Renewed token of kind {}, new lifetime:{}", LOG.info("Renewed token of kind {}, new lifetime:{}",
token.getKind(), newTokenLife); token.getKind(), newTokenLife);
Assert.assertTrue(newTokenLife > tokenLife); Assert.assertTrue(newTokenLife > tokenLife);
@ -1819,25 +1846,34 @@ public class TestKMS {
// test delegation token cancellation // test delegation token cancellation
for (Token<?> token : tokens) { for (Token<?> token : tokens) {
if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) { if (!(token.getKind()
.equals(KMSClientProvider.TOKEN_KIND))) {
LOG.info("Skipping token {}", token); LOG.info("Skipping token {}", token);
continue; continue;
} }
LOG.info("Got dt for " + uri + "; " + token); LOG.info("Got dt for " + uri + "; " + token);
token.cancel(conf); token.cancel(clientConf);
LOG.info("Cancelled token of kind {}", token.getKind()); LOG.info("Cancelled token of kind {}", token.getKind());
doNothing().when(mockAuthenticator).
authenticate(any(URL.class), any(AuthenticatedURL.Token.class));
try { try {
token.renew(conf); token.renew(clientConf);
Assert.fail("should not be able to renew a canceled token"); Assert
.fail("should not be able to renew a canceled token");
} catch (Exception e) { } catch (Exception e) {
LOG.info("Expected exception when trying to renew token", e); LOG.info("Expected exception when renewing token", e);
} }
} }
return null; return null;
} }
}); });
return null;
} finally {
otherUgi.logoutUserFromKeytab();
}
}
});
return null;
}
});
} }
@Test @Test