|
@@ -35,6 +35,7 @@ import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
|
|
import org.apache.hadoop.crypto.key.kms.ValueQueue;
|
|
import org.apache.hadoop.crypto.key.kms.ValueQueue;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
+import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.MultipleIOException;
|
|
import org.apache.hadoop.io.MultipleIOException;
|
|
import org.apache.hadoop.minikdc.MiniKdc;
|
|
import org.apache.hadoop.minikdc.MiniKdc;
|
|
import org.apache.hadoop.security.Credentials;
|
|
import org.apache.hadoop.security.Credentials;
|
|
@@ -44,6 +45,7 @@ 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.ssl.SSLFactory;
|
|
import org.apache.hadoop.security.ssl.SSLFactory;
|
|
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.delegation.web.DelegationTokenIdentifier;
|
|
import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.Whitebox;
|
|
import org.apache.hadoop.test.Whitebox;
|
|
@@ -96,6 +98,8 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|
import java.util.regex.Matcher;
|
|
import java.util.regex.Matcher;
|
|
import java.util.regex.Pattern;
|
|
import java.util.regex.Pattern;
|
|
|
|
|
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH;
|
|
|
|
+
|
|
import static org.junit.Assert.assertArrayEquals;
|
|
import static org.junit.Assert.assertArrayEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertFalse;
|
|
@@ -141,21 +145,78 @@ public class TestKMS {
|
|
}
|
|
}
|
|
|
|
|
|
public static abstract class KMSCallable<T> implements Callable<T> {
|
|
public static abstract class KMSCallable<T> implements Callable<T> {
|
|
- private URL kmsUrl;
|
|
|
|
|
|
+ private List<URL> kmsUrl;
|
|
|
|
|
|
protected URL getKMSUrl() {
|
|
protected URL getKMSUrl() {
|
|
- return kmsUrl;
|
|
|
|
|
|
+ return kmsUrl.get(0);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected URL[] getKMSHAUrl() {
|
|
|
|
+ URL[] urls = new URL[kmsUrl.size()];
|
|
|
|
+ return kmsUrl.toArray(urls);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected void addKMSUrl(URL url) {
|
|
|
|
+ if (kmsUrl == null) {
|
|
|
|
+ kmsUrl = new ArrayList<URL>();
|
|
|
|
+ }
|
|
|
|
+ kmsUrl.add(url);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /*
|
|
|
|
+ * The format of the returned value will be
|
|
|
|
+ * kms://http:kms1.example1.com:port1,kms://http:kms2.example2.com:port2
|
|
|
|
+ */
|
|
|
|
+ protected String generateLoadBalancingKeyProviderUriString() {
|
|
|
|
+ if (kmsUrl == null || kmsUrl.size() == 0) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ StringBuffer sb = new StringBuffer();
|
|
|
|
+
|
|
|
|
+ for (int i = 0; i < kmsUrl.size(); i++) {
|
|
|
|
+ sb.append(KMSClientProvider.SCHEME_NAME + "://" +
|
|
|
|
+ kmsUrl.get(0).getProtocol() + "@");
|
|
|
|
+ URL url = kmsUrl.get(i);
|
|
|
|
+ sb.append(url.getAuthority());
|
|
|
|
+ if (url.getPath() != null) {
|
|
|
|
+ sb.append(url.getPath());
|
|
|
|
+ }
|
|
|
|
+ if (i < kmsUrl.size() - 1) {
|
|
|
|
+ sb.append(",");
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return sb.toString();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
protected KeyProvider createProvider(URI uri, Configuration conf)
|
|
protected KeyProvider createProvider(URI uri, Configuration conf)
|
|
throws IOException {
|
|
throws IOException {
|
|
- final KeyProvider ret = new LoadBalancingKMSClientProvider(
|
|
|
|
|
|
+ final KeyProvider ret = new LoadBalancingKMSClientProvider(uri,
|
|
new KMSClientProvider[] {new KMSClientProvider(uri, conf)}, conf);
|
|
new KMSClientProvider[] {new KMSClientProvider(uri, conf)}, conf);
|
|
providersCreated.add(ret);
|
|
providersCreated.add(ret);
|
|
return ret;
|
|
return ret;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * create a LoadBalancingKMSClientProvider from an array of URIs.
|
|
|
|
+ * @param uris an array of KMS URIs
|
|
|
|
+ * @param conf configuration object
|
|
|
|
+ * @return a LoadBalancingKMSClientProvider object
|
|
|
|
+ * @throws IOException
|
|
|
|
+ */
|
|
|
|
+ protected LoadBalancingKMSClientProvider createHAProvider(URI lbUri,
|
|
|
|
+ URI[] uris, Configuration conf) throws IOException {
|
|
|
|
+ KMSClientProvider[] providers = new KMSClientProvider[uris.length];
|
|
|
|
+ for (int i = 0; i < providers.length; i++) {
|
|
|
|
+ providers[i] =
|
|
|
|
+ new KMSClientProvider(uris[i], conf);
|
|
|
|
+ }
|
|
|
|
+ final LoadBalancingKMSClientProvider ret =
|
|
|
|
+ new LoadBalancingKMSClientProvider(lbUri, providers, conf);
|
|
|
|
+ providersCreated.add(ret);
|
|
|
|
+ return ret;
|
|
|
|
+ }
|
|
|
|
+
|
|
private KMSClientProvider createKMSClientProvider(URI uri, Configuration conf)
|
|
private KMSClientProvider createKMSClientProvider(URI uri, Configuration conf)
|
|
throws IOException {
|
|
throws IOException {
|
|
final KMSClientProvider ret = new KMSClientProvider(uri, conf);
|
|
final KMSClientProvider ret = new KMSClientProvider(uri, conf);
|
|
@@ -170,22 +231,34 @@ public class TestKMS {
|
|
|
|
|
|
protected <T> T runServer(int port, String keystore, String password, File confDir,
|
|
protected <T> T runServer(int port, String keystore, String password, File confDir,
|
|
KMSCallable<T> callable) throws Exception {
|
|
KMSCallable<T> callable) throws Exception {
|
|
|
|
+ return runServer(new int[] {port}, keystore, password, confDir, callable);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected <T> T runServer(int[] ports, String keystore, String password,
|
|
|
|
+ File confDir, KMSCallable<T> callable) throws Exception {
|
|
MiniKMS.Builder miniKMSBuilder = new MiniKMS.Builder().setKmsConfDir(confDir)
|
|
MiniKMS.Builder miniKMSBuilder = new MiniKMS.Builder().setKmsConfDir(confDir)
|
|
.setLog4jConfFile("log4j.properties");
|
|
.setLog4jConfFile("log4j.properties");
|
|
if (keystore != null) {
|
|
if (keystore != null) {
|
|
miniKMSBuilder.setSslConf(new File(keystore), password);
|
|
miniKMSBuilder.setSslConf(new File(keystore), password);
|
|
}
|
|
}
|
|
- if (port > 0) {
|
|
|
|
- miniKMSBuilder.setPort(port);
|
|
|
|
|
|
+ final List<MiniKMS> kmsList = new ArrayList<>();
|
|
|
|
+
|
|
|
|
+ for (int i = 0; i < ports.length; i++) {
|
|
|
|
+ if (ports[i] > 0) {
|
|
|
|
+ miniKMSBuilder.setPort(ports[i]);
|
|
|
|
+ }
|
|
|
|
+ MiniKMS miniKMS = miniKMSBuilder.build();
|
|
|
|
+ kmsList.add(miniKMS);
|
|
|
|
+ miniKMS.start();
|
|
|
|
+ LOG.info("Test KMS running at: " + miniKMS.getKMSUrl());
|
|
|
|
+ callable.addKMSUrl(miniKMS.getKMSUrl());
|
|
}
|
|
}
|
|
- MiniKMS miniKMS = miniKMSBuilder.build();
|
|
|
|
- miniKMS.start();
|
|
|
|
try {
|
|
try {
|
|
- System.out.println("Test KMS running at: " + miniKMS.getKMSUrl());
|
|
|
|
- callable.kmsUrl = miniKMS.getKMSUrl();
|
|
|
|
return callable.call();
|
|
return callable.call();
|
|
} finally {
|
|
} finally {
|
|
- miniKMS.stop();
|
|
|
|
|
|
+ for (MiniKMS miniKMS: kmsList) {
|
|
|
|
+ miniKMS.stop();
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -240,6 +313,13 @@ public class TestKMS {
|
|
return new URI("kms://" + str);
|
|
return new URI("kms://" + str);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public static URI[] createKMSHAUri(URL[] kmsUrls) throws Exception {
|
|
|
|
+ URI[] uris = new URI[kmsUrls.length];
|
|
|
|
+ for (int i=0; i< kmsUrls.length; i++) {
|
|
|
|
+ uris[i] = createKMSUri(kmsUrls[i]);
|
|
|
|
+ }
|
|
|
|
+ return uris;
|
|
|
|
+ }
|
|
|
|
|
|
private static class KerberosConfiguration
|
|
private static class KerberosConfiguration
|
|
extends javax.security.auth.login.Configuration {
|
|
extends javax.security.auth.login.Configuration {
|
|
@@ -306,6 +386,7 @@ public class TestKMS {
|
|
principals.add("otheradmin");
|
|
principals.add("otheradmin");
|
|
principals.add("client/host");
|
|
principals.add("client/host");
|
|
principals.add("client1");
|
|
principals.add("client1");
|
|
|
|
+ principals.add("foo");
|
|
for (KMSACLs.Type type : KMSACLs.Type.values()) {
|
|
for (KMSACLs.Type type : KMSACLs.Type.values()) {
|
|
principals.add(type.toString());
|
|
principals.add(type.toString());
|
|
}
|
|
}
|
|
@@ -2011,7 +2092,6 @@ public class TestKMS {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
});
|
|
});
|
|
-
|
|
|
|
nonKerberosUgi.addCredentials(credentials);
|
|
nonKerberosUgi.addCredentials(credentials);
|
|
|
|
|
|
try {
|
|
try {
|
|
@@ -2067,6 +2147,18 @@ public class TestKMS {
|
|
testDelegationTokensOps(true, true);
|
|
testDelegationTokensOps(true, true);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private Text getTokenService(KeyProvider provider) {
|
|
|
|
+ assertTrue("KeyProvider should be an instance of " +
|
|
|
|
+ "LoadBalancingKMSClientProvider", (provider instanceof
|
|
|
|
+ LoadBalancingKMSClientProvider));
|
|
|
|
+ assertEquals("Num client providers should be 1", 1,
|
|
|
|
+ ((LoadBalancingKMSClientProvider)provider).getProviders().length);
|
|
|
|
+ final Text tokenService = new Text(
|
|
|
|
+ (((LoadBalancingKMSClientProvider)provider).getProviders()[0])
|
|
|
|
+ .getCanonicalServiceName());
|
|
|
|
+ return tokenService;
|
|
|
|
+ }
|
|
|
|
+
|
|
private void testDelegationTokensOps(final boolean ssl, final boolean kerb)
|
|
private void testDelegationTokensOps(final boolean ssl, final boolean kerb)
|
|
throws Exception {
|
|
throws Exception {
|
|
final File confDir = getTestDir();
|
|
final File confDir = getTestDir();
|
|
@@ -2103,6 +2195,10 @@ public class TestKMS {
|
|
@Override
|
|
@Override
|
|
public Void run() throws Exception {
|
|
public Void run() throws Exception {
|
|
KeyProvider kp = createProvider(uri, clientConf);
|
|
KeyProvider kp = createProvider(uri, clientConf);
|
|
|
|
+ // Unset the conf value for key provider path just to be sure that
|
|
|
|
+ // the key provider created for renew and cancel token is from
|
|
|
|
+ // token service field.
|
|
|
|
+ clientConf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
|
|
// test delegation token retrieval
|
|
// test delegation token retrieval
|
|
KeyProviderDelegationTokenExtension kpdte =
|
|
KeyProviderDelegationTokenExtension kpdte =
|
|
KeyProviderDelegationTokenExtension.
|
|
KeyProviderDelegationTokenExtension.
|
|
@@ -2110,13 +2206,10 @@ public class TestKMS {
|
|
final Credentials credentials = new Credentials();
|
|
final Credentials credentials = new Credentials();
|
|
final Token<?>[] tokens =
|
|
final Token<?>[] tokens =
|
|
kpdte.addDelegationTokens("client1", credentials);
|
|
kpdte.addDelegationTokens("client1", credentials);
|
|
|
|
+ Text tokenService = getTokenService(kp);
|
|
Assert.assertEquals(1, credentials.getAllTokens().size());
|
|
Assert.assertEquals(1, credentials.getAllTokens().size());
|
|
- InetSocketAddress kmsAddr =
|
|
|
|
- new InetSocketAddress(getKMSUrl().getHost(),
|
|
|
|
- getKMSUrl().getPort());
|
|
|
|
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
|
|
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
|
|
- credentials.getToken(SecurityUtil.buildTokenService(kmsAddr)).
|
|
|
|
- getKind());
|
|
|
|
|
|
+ credentials.getToken(tokenService).getKind());
|
|
|
|
|
|
// Test non-renewer user cannot renew.
|
|
// Test non-renewer user cannot renew.
|
|
for (Token<?> token : tokens) {
|
|
for (Token<?> token : tokens) {
|
|
@@ -2258,15 +2351,16 @@ public class TestKMS {
|
|
// Get a DT and use it.
|
|
// Get a DT and use it.
|
|
final Credentials credentials = new Credentials();
|
|
final Credentials credentials = new Credentials();
|
|
kpdte.addDelegationTokens("client", credentials);
|
|
kpdte.addDelegationTokens("client", credentials);
|
|
|
|
+ Text tokenService = getTokenService(kp);
|
|
Assert.assertEquals(1, credentials.getAllTokens().size());
|
|
Assert.assertEquals(1, credentials.getAllTokens().size());
|
|
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND, credentials.
|
|
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND, credentials.
|
|
- getToken(SecurityUtil.buildTokenService(kmsAddr)).getKind());
|
|
|
|
|
|
+ getToken(tokenService).getKind());
|
|
UserGroupInformation.getCurrentUser().addCredentials(credentials);
|
|
UserGroupInformation.getCurrentUser().addCredentials(credentials);
|
|
LOG.info("Added kms dt to credentials: {}", UserGroupInformation.
|
|
LOG.info("Added kms dt to credentials: {}", UserGroupInformation.
|
|
getCurrentUser().getCredentials().getAllTokens());
|
|
getCurrentUser().getCredentials().getAllTokens());
|
|
Token<?> token =
|
|
Token<?> token =
|
|
UserGroupInformation.getCurrentUser().getCredentials()
|
|
UserGroupInformation.getCurrentUser().getCredentials()
|
|
- .getToken(SecurityUtil.buildTokenService(kmsAddr));
|
|
|
|
|
|
+ .getToken(tokenService);
|
|
Assert.assertNotNull(token);
|
|
Assert.assertNotNull(token);
|
|
job1Token.add(token);
|
|
job1Token.add(token);
|
|
|
|
|
|
@@ -2302,17 +2396,17 @@ public class TestKMS {
|
|
// Get a new DT, but don't use it yet.
|
|
// Get a new DT, but don't use it yet.
|
|
final Credentials newCreds = new Credentials();
|
|
final Credentials newCreds = new Credentials();
|
|
kpdte.addDelegationTokens("client", newCreds);
|
|
kpdte.addDelegationTokens("client", newCreds);
|
|
|
|
+ Text tokenService = getTokenService(kp);
|
|
Assert.assertEquals(1, newCreds.getAllTokens().size());
|
|
Assert.assertEquals(1, newCreds.getAllTokens().size());
|
|
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
|
|
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
|
|
- newCreds.getToken(SecurityUtil.buildTokenService(kmsAddr)).
|
|
|
|
|
|
+ newCreds.getToken(tokenService).
|
|
getKind());
|
|
getKind());
|
|
|
|
|
|
// Using job 1's DT should fail.
|
|
// Using job 1's DT should fail.
|
|
final Credentials oldCreds = new Credentials();
|
|
final Credentials oldCreds = new Credentials();
|
|
for (Token<?> token : job1Token) {
|
|
for (Token<?> token : job1Token) {
|
|
if (token.getKind().equals(KMSDelegationToken.TOKEN_KIND)) {
|
|
if (token.getKind().equals(KMSDelegationToken.TOKEN_KIND)) {
|
|
- oldCreds
|
|
|
|
- .addToken(SecurityUtil.buildTokenService(kmsAddr), token);
|
|
|
|
|
|
+ oldCreds.addToken(tokenService, token);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
UserGroupInformation.getCurrentUser().addCredentials(oldCreds);
|
|
UserGroupInformation.getCurrentUser().addCredentials(oldCreds);
|
|
@@ -2328,7 +2422,7 @@ public class TestKMS {
|
|
// Using the new DT should succeed.
|
|
// Using the new DT should succeed.
|
|
Assert.assertEquals(1, newCreds.getAllTokens().size());
|
|
Assert.assertEquals(1, newCreds.getAllTokens().size());
|
|
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
|
|
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
|
|
- newCreds.getToken(SecurityUtil.buildTokenService(kmsAddr)).
|
|
|
|
|
|
+ newCreds.getToken(tokenService).
|
|
getKind());
|
|
getKind());
|
|
UserGroupInformation.getCurrentUser().addCredentials(newCreds);
|
|
UserGroupInformation.getCurrentUser().addCredentials(newCreds);
|
|
LOG.info("Credetials now are: {}", UserGroupInformation
|
|
LOG.info("Credetials now are: {}", UserGroupInformation
|
|
@@ -2357,7 +2451,14 @@ public class TestKMS {
|
|
doKMSWithZK(true, true);
|
|
doKMSWithZK(true, true);
|
|
}
|
|
}
|
|
|
|
|
|
- public void doKMSWithZK(boolean zkDTSM, boolean zkSigner) throws Exception {
|
|
|
|
|
|
+ private <T> T runServerWithZooKeeper(boolean zkDTSM, boolean zkSigner,
|
|
|
|
+ KMSCallable<T> callable) throws Exception {
|
|
|
|
+ return runServerWithZooKeeper(zkDTSM, zkSigner, callable, 1);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private <T> T runServerWithZooKeeper(boolean zkDTSM, boolean zkSigner,
|
|
|
|
+ KMSCallable<T> callable, int kmsSize) throws Exception {
|
|
|
|
+
|
|
TestingServer zkServer = null;
|
|
TestingServer zkServer = null;
|
|
try {
|
|
try {
|
|
zkServer = new TestingServer();
|
|
zkServer = new TestingServer();
|
|
@@ -2403,43 +2504,189 @@ public class TestKMS {
|
|
|
|
|
|
writeConf(testDir, conf);
|
|
writeConf(testDir, conf);
|
|
|
|
|
|
- KMSCallable<KeyProvider> c =
|
|
|
|
- new KMSCallable<KeyProvider>() {
|
|
|
|
- @Override
|
|
|
|
- public KeyProvider call() throws Exception {
|
|
|
|
- final Configuration conf = new Configuration();
|
|
|
|
- conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
|
|
|
|
- final URI uri = createKMSUri(getKMSUrl());
|
|
|
|
-
|
|
|
|
- final KeyProvider kp =
|
|
|
|
- doAs("SET_KEY_MATERIAL",
|
|
|
|
- new PrivilegedExceptionAction<KeyProvider>() {
|
|
|
|
- @Override
|
|
|
|
- public KeyProvider run() throws Exception {
|
|
|
|
- KeyProvider kp = createProvider(uri, conf);
|
|
|
|
- kp.createKey("k1", new byte[16],
|
|
|
|
- new KeyProvider.Options(conf));
|
|
|
|
- kp.createKey("k2", new byte[16],
|
|
|
|
- new KeyProvider.Options(conf));
|
|
|
|
- kp.createKey("k3", new byte[16],
|
|
|
|
- new KeyProvider.Options(conf));
|
|
|
|
- return kp;
|
|
|
|
- }
|
|
|
|
- });
|
|
|
|
- return kp;
|
|
|
|
- }
|
|
|
|
- };
|
|
|
|
-
|
|
|
|
- runServer(null, null, testDir, c);
|
|
|
|
|
|
+ int[] ports = new int[kmsSize];
|
|
|
|
+ for (int i = 0; i < ports.length; i++) {
|
|
|
|
+ ports[i] = -1;
|
|
|
|
+ }
|
|
|
|
+ return runServer(ports, null, null, testDir, callable);
|
|
} finally {
|
|
} finally {
|
|
if (zkServer != null) {
|
|
if (zkServer != null) {
|
|
zkServer.stop();
|
|
zkServer.stop();
|
|
zkServer.close();
|
|
zkServer.close();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ public void doKMSWithZK(boolean zkDTSM, boolean zkSigner) throws Exception {
|
|
|
|
+ KMSCallable<KeyProvider> c =
|
|
|
|
+ new KMSCallable<KeyProvider>() {
|
|
|
|
+ @Override
|
|
|
|
+ public KeyProvider call() throws Exception {
|
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
|
+ conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
|
|
|
|
+ final URI uri = createKMSUri(getKMSUrl());
|
|
|
|
+
|
|
|
|
+ final KeyProvider kp =
|
|
|
|
+ doAs("SET_KEY_MATERIAL",
|
|
|
|
+ new PrivilegedExceptionAction<KeyProvider>() {
|
|
|
|
+ @Override
|
|
|
|
+ public KeyProvider run() throws Exception {
|
|
|
|
+ KeyProvider kp = createProvider(uri, conf);
|
|
|
|
+ kp.createKey("k1", new byte[16],
|
|
|
|
+ new KeyProvider.Options(conf));
|
|
|
|
+ kp.createKey("k2", new byte[16],
|
|
|
|
+ new KeyProvider.Options(conf));
|
|
|
|
+ kp.createKey("k3", new byte[16],
|
|
|
|
+ new KeyProvider.Options(conf));
|
|
|
|
+ return kp;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ return kp;
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+
|
|
|
|
+ runServerWithZooKeeper(zkDTSM, zkSigner, c);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testKMSHAZooKeeperDelegationToken() throws Exception {
|
|
|
|
+ final int kmsSize = 2;
|
|
|
|
+ doKMSWithZKWithDelegationToken(true, true, kmsSize);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private void doKMSWithZKWithDelegationToken(boolean zkDTSM, boolean zkSigner,
|
|
|
|
+ int kmsSize) throws Exception {
|
|
|
|
+ // Create a KMSCallable to execute requests after ZooKeeper and KMS are up.
|
|
|
|
+ KMSCallable<Void> c = new KMSCallable<Void>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Void call() throws Exception {
|
|
|
|
+ final Configuration conf = new Configuration();
|
|
|
|
+ conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
|
|
|
|
+ final URI[] uris = createKMSHAUri(getKMSHAUrl());
|
|
|
|
+ final Credentials credentials = new Credentials();
|
|
|
|
+ // Create a UGI without Kerberos auth. It will authenticate with tokens.
|
|
|
|
+ final UserGroupInformation nonKerberosUgi =
|
|
|
|
+ UserGroupInformation.getCurrentUser();
|
|
|
|
+ final String lbUri = generateLoadBalancingKeyProviderUriString();
|
|
|
|
+ final LoadBalancingKMSClientProvider lbkp =
|
|
|
|
+ createHAProvider(URI.create(lbUri), uris, conf);
|
|
|
|
+ conf.unset(HADOOP_SECURITY_KEY_PROVIDER_PATH);
|
|
|
|
+ // get delegation tokens using kerberos login
|
|
|
|
+ doAs("SET_KEY_MATERIAL",
|
|
|
|
+ new PrivilegedExceptionAction<Void>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Void run() throws Exception {
|
|
|
|
+ KeyProviderDelegationTokenExtension kpdte =
|
|
|
|
+ KeyProviderDelegationTokenExtension.
|
|
|
|
+ createKeyProviderDelegationTokenExtension(lbkp);
|
|
|
|
+ kpdte.addDelegationTokens("foo", credentials);
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+
|
|
|
|
+ nonKerberosUgi.addCredentials(credentials);
|
|
|
|
+ // Access KMS using delegation token for authentication, no Kerberos.
|
|
|
|
+ nonKerberosUgi.doAs(new PrivilegedExceptionAction<Void>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Void run() throws Exception {
|
|
|
|
+ // Create a kms client with one provider at a time. Must use one
|
|
|
|
+ // provider so that if it fails to authenticate, it does not fall
|
|
|
|
+ // back to the next KMS instance.
|
|
|
|
+ // Should succeed because it has delegation tokens for any instance.
|
|
|
|
+ int i = 0;
|
|
|
|
+ for (KMSClientProvider provider : lbkp.getProviders()) {
|
|
|
|
+ final String key = "k" + i++;
|
|
|
|
+ LOG.info("Connect to {} to create key {}.", provider, key);
|
|
|
|
+ provider.createKey(key, new KeyProvider.Options(conf));
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+
|
|
|
|
+ final Collection<Token<? extends TokenIdentifier>> tokens =
|
|
|
|
+ credentials.getAllTokens();
|
|
|
|
+ doAs("foo", new PrivilegedExceptionAction<Void>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Void run() throws Exception {
|
|
|
|
+ assertEquals(1, tokens.size());
|
|
|
|
+ Token token = tokens.iterator().next();
|
|
|
|
+ assertEquals(KMSDelegationToken.TOKEN_KIND, token.getKind());
|
|
|
|
+ LOG.info("Got dt for token: {}", token);
|
|
|
|
+ final long tokenLife = token.renew(conf);
|
|
|
|
+ LOG.info("Renewed token {}, new lifetime:{}", token, tokenLife);
|
|
|
|
+ Thread.sleep(10);
|
|
|
|
+ final long newTokenLife = token.renew(conf);
|
|
|
|
+ LOG.info("Renewed token {}, new lifetime:{}", token, newTokenLife);
|
|
|
|
+ assertTrue(newTokenLife > tokenLife);
|
|
|
|
+
|
|
|
|
+ // test delegation token cancellation
|
|
|
|
+ LOG.info("Got dt for token: {}", token);
|
|
|
|
+ token.cancel(conf);
|
|
|
|
+ LOG.info("Cancelled token {}", token);
|
|
|
|
+ try {
|
|
|
|
+ token.renew(conf);
|
|
|
|
+ fail("should not be able to renew a canceled token");
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.info("Expected exception when renewing token", e);
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+
|
|
|
|
+ final Credentials newCredentials = new Credentials();
|
|
|
|
+ doAs("SET_KEY_MATERIAL",
|
|
|
|
+ new PrivilegedExceptionAction<Void>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Void run() throws Exception {
|
|
|
|
+ KeyProviderDelegationTokenExtension kpdte =
|
|
|
|
+ KeyProviderDelegationTokenExtension.
|
|
|
|
+ createKeyProviderDelegationTokenExtension(lbkp);
|
|
|
|
+ kpdte.addDelegationTokens("foo", newCredentials);
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+
|
|
|
|
+ doAs("foo", new PrivilegedExceptionAction<Void>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Void run() throws Exception {
|
|
|
|
+ KMSClientProvider kp1 = lbkp.getProviders()[0];
|
|
|
|
+ URL[] urls = getKMSHAUrl();
|
|
|
|
+ final Collection<Token<? extends TokenIdentifier>> tokens =
|
|
|
|
+ newCredentials.getAllTokens();
|
|
|
|
+ assertEquals(1, tokens.size());
|
|
|
|
+ Token token = tokens.iterator().next();
|
|
|
|
+ assertEquals(KMSDelegationToken.TOKEN_KIND,
|
|
|
|
+ token.getKind());
|
|
|
|
+ // Testing backward compatibility of token renewal and cancellation.
|
|
|
|
+ // Set the token service to ip:port format and test to renew/cancel.
|
|
|
|
+ Text text = SecurityUtil.buildTokenService(
|
|
|
|
+ new InetSocketAddress(urls[0].getHost(), urls[0].getPort()));
|
|
|
|
+ token.setService(text);
|
|
|
|
+ conf.set(HADOOP_SECURITY_KEY_PROVIDER_PATH, lbUri);
|
|
|
|
+ long tokenLife = 0L;
|
|
|
|
+ for (KMSClientProvider kp : lbkp.getProviders()) {
|
|
|
|
+ long renewedTokenLife = token.renew(conf);
|
|
|
|
+ LOG.info("Renewed token of kind {}, new lifetime:{}",
|
|
|
|
+ token.getKind(), renewedTokenLife);
|
|
|
|
+ assertTrue(renewedTokenLife > tokenLife);
|
|
|
|
+ tokenLife = renewedTokenLife;
|
|
|
|
+ Thread.sleep(10);
|
|
|
|
+ }
|
|
|
|
+ token.cancel(conf);
|
|
|
|
+ try {
|
|
|
|
+ token.renew(conf);
|
|
|
|
+ fail("should not be able to renew a canceled token");
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.info("Expected exception when renewing token", e);
|
|
|
|
+ }
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+ runServerWithZooKeeper(zkDTSM, zkSigner, c, kmsSize);
|
|
|
|
+ }
|
|
|
|
|
|
@Test
|
|
@Test
|
|
public void testProxyUserKerb() throws Exception {
|
|
public void testProxyUserKerb() throws Exception {
|