|
@@ -19,7 +19,7 @@ package org.apache.hadoop.crypto.key.kms.server;
|
|
|
|
|
|
import org.apache.curator.test.TestingServer;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.crypto.key.kms.server.KeyAuthorizationKeyProvider;
|
|
|
+import org.apache.hadoop.crypto.key.KeyProviderFactory;
|
|
|
import org.apache.hadoop.crypto.key.KeyProvider;
|
|
|
import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
|
|
|
import org.apache.hadoop.crypto.key.KeyProvider.Options;
|
|
@@ -31,11 +31,14 @@ import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
|
|
|
import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.minikdc.MiniKdc;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
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.ssl.KeyStoreTestUtil;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
@@ -45,11 +48,12 @@ import org.junit.AfterClass;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.BeforeClass;
|
|
|
+import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
+import org.junit.rules.Timeout;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import javax.security.auth.kerberos.KerberosPrincipal;
|
|
|
import javax.security.auth.login.AppConfigurationEntry;
|
|
|
|
|
|
import java.io.File;
|
|
@@ -72,9 +76,17 @@ import java.util.Properties;
|
|
|
import java.util.UUID;
|
|
|
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 {
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(TestKMS.class);
|
|
|
|
|
|
+ @Rule
|
|
|
+ public final Timeout testTimeout = new Timeout(180000);
|
|
|
+
|
|
|
@Before
|
|
|
public void cleanUp() {
|
|
|
// resetting kerberos security
|
|
@@ -649,20 +661,6 @@ public class TestKMS {
|
|
|
Assert.assertEquals("d", meta.getDescription());
|
|
|
Assert.assertEquals(attributes, meta.getAttributes());
|
|
|
|
|
|
- // test delegation token retrieval
|
|
|
- KeyProviderDelegationTokenExtension kpdte =
|
|
|
- KeyProviderDelegationTokenExtension.
|
|
|
- createKeyProviderDelegationTokenExtension(kp);
|
|
|
- Credentials credentials = new Credentials();
|
|
|
- kpdte.addDelegationTokens("foo", credentials);
|
|
|
- Assert.assertEquals(1, credentials.getAllTokens().size());
|
|
|
- InetSocketAddress kmsAddr = new InetSocketAddress(getKMSUrl().getHost(),
|
|
|
- getKMSUrl().getPort());
|
|
|
-
|
|
|
- Assert.assertEquals(new Text("kms-dt"), credentials.getToken(
|
|
|
- SecurityUtil.buildTokenService(kmsAddr)).getKind());
|
|
|
-
|
|
|
-
|
|
|
// test rollover draining
|
|
|
KeyProviderCryptoExtension kpce = KeyProviderCryptoExtension.
|
|
|
createKeyProviderCryptoExtension(kp);
|
|
@@ -1745,6 +1743,101 @@ public class TestKMS {
|
|
|
});
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testDelegationTokensOpsSimple() throws Exception {
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
+ final Authenticator mock = mock(PseudoAuthenticator.class);
|
|
|
+ testDelegationTokensOps(conf, mock);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testDelegationTokensOpsKerberized() throws Exception {
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
+ conf.set("hadoop.security.authentication", "kerberos");
|
|
|
+ final Authenticator mock = mock(KerberosAuthenticator.class);
|
|
|
+ testDelegationTokensOps(conf, mock);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testDelegationTokensOps(Configuration conf,
|
|
|
+ final Authenticator mockAuthenticator) throws Exception {
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
|
+ File confDir = getTestDir();
|
|
|
+ conf = createBaseKMSConf(confDir);
|
|
|
+ writeConf(confDir, conf);
|
|
|
+ doNothing().when(mockAuthenticator).authenticate(any(URL.class),
|
|
|
+ any(AuthenticatedURL.Token.class));
|
|
|
+
|
|
|
+ runServer(null, null, confDir, new KMSCallable<Void>() {
|
|
|
+ @Override
|
|
|
+ public Void call() throws Exception {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ URI uri = createKMSUri(getKMSUrl());
|
|
|
+ KeyProvider kp = createProvider(uri, conf);
|
|
|
+ conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
|
|
|
+ createKMSUri(getKMSUrl()).toString());
|
|
|
+
|
|
|
+ // test delegation token retrieval
|
|
|
+ KeyProviderDelegationTokenExtension kpdte =
|
|
|
+ KeyProviderDelegationTokenExtension.
|
|
|
+ createKeyProviderDelegationTokenExtension(kp);
|
|
|
+ Credentials credentials = new Credentials();
|
|
|
+ final Token<?>[] tokens = kpdte.addDelegationTokens(
|
|
|
+ UserGroupInformation.getCurrentUser().getUserName(), credentials);
|
|
|
+ Assert.assertEquals(1, credentials.getAllTokens().size());
|
|
|
+ InetSocketAddress kmsAddr = new InetSocketAddress(getKMSUrl().getHost(),
|
|
|
+ getKMSUrl().getPort());
|
|
|
+ Assert.assertEquals(KMSClientProvider.TOKEN_KIND,
|
|
|
+ credentials.getToken(SecurityUtil.buildTokenService(kmsAddr)).
|
|
|
+ getKind());
|
|
|
+
|
|
|
+ // After this point, we're supposed to use the delegation token to auth.
|
|
|
+ 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) {
|
|
|
+ if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) {
|
|
|
+ LOG.info("Skipping token {}", token);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ LOG.info("Got dt for " + uri + "; " + token);
|
|
|
+ long tokenLife = token.renew(conf);
|
|
|
+ LOG.info("Renewed token of kind {}, new lifetime:{}",
|
|
|
+ token.getKind(), tokenLife);
|
|
|
+ Thread.sleep(100);
|
|
|
+ long newTokenLife = token.renew(conf);
|
|
|
+ LOG.info("Renewed token of kind {}, new lifetime:{}",
|
|
|
+ token.getKind(), newTokenLife);
|
|
|
+ Assert.assertTrue(newTokenLife > tokenLife);
|
|
|
+ renewed = true;
|
|
|
+ }
|
|
|
+ Assert.assertTrue(renewed);
|
|
|
+
|
|
|
+ // test delegation token cancellation
|
|
|
+ for (Token<?> token : tokens) {
|
|
|
+ if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) {
|
|
|
+ LOG.info("Skipping token {}", token);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ LOG.info("Got dt for " + uri + "; " + token);
|
|
|
+ token.cancel(conf);
|
|
|
+ LOG.info("Cancelled token of kind {}", token.getKind());
|
|
|
+ doNothing().when(mockAuthenticator).
|
|
|
+ authenticate(any(URL.class), any(AuthenticatedURL.Token.class));
|
|
|
+ try {
|
|
|
+ token.renew(conf);
|
|
|
+ Assert.fail("should not be able to renew a canceled token");
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.info("Expected exception when trying to renew token", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testKMSWithZKSigner() throws Exception {
|
|
|
doKMSWithZK(true, false);
|