You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2016/06/28 01:30:06 UTC

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

Repository: hadoop
Updated Branches:
  refs/heads/trunk 9683eab0e -> 771f798ed


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


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/771f798e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/771f798e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/771f798e

Branch: refs/heads/trunk
Commit: 771f798edf97b27ae003395118c0317b484df6ee
Parents: 9683eab
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Jun 27 18:20:56 2016 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Jun 27 18:20:56 2016 -0700

----------------------------------------------------------------------
 .../hadoop/security/UserGroupInformation.java   |   2 +-
 .../web/DelegationTokenAuthenticator.java       |  55 ++++--
 .../delegation/web/DelegationTokenManager.java  |   7 +
 .../hadoop-kms/src/site/markdown/index.md.vm    |   2 +-
 .../hadoop/crypto/key/kms/server/TestKMS.java   | 186 +++++++++++--------
 5 files changed, 155 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/771f798e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
index 93822a1..7dee14b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
@@ -339,7 +339,7 @@ public class UserGroupInformation {
   
   @InterfaceAudience.Private
   @VisibleForTesting
-  static void reset() {
+  public static void reset() {
     authenticationMethod = null;
     conf = null;
     groups = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771f798e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
index 37ae601..2d60d4a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java
@@ -58,6 +58,7 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
   private static final String HTTP_PUT = "PUT";
 
   public static final String OP_PARAM = "op";
+  private static final String OP_PARAM_EQUALS = OP_PARAM + "=";
 
   public static final String DELEGATION_TOKEN_HEADER =
       "X-Hadoop-Delegation-Token";
@@ -285,27 +286,41 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     }
     url = new URL(sb.toString());
     AuthenticatedURL aUrl = new AuthenticatedURL(this, connConfigurator);
-    HttpURLConnection conn = aUrl.openConnection(url, token);
-    conn.setRequestMethod(operation.getHttpMethod());
-    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
-    if (hasResponse) {
-      String contentType = conn.getHeaderField(CONTENT_TYPE);
-      contentType = (contentType != null) ? StringUtils.toLowerCase(contentType)
-                                          : null;
-      if (contentType != null &&
-          contentType.contains(APPLICATION_JSON_MIME)) {
-        try {
-          ObjectMapper mapper = new ObjectMapper();
-          ret = mapper.readValue(conn.getInputStream(), Map.class);
-        } catch (Exception ex) {
-          throw new AuthenticationException(String.format(
-              "'%s' did not handle the '%s' delegation token operation: %s",
-              url.getAuthority(), operation, ex.getMessage()), ex);
+    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);
+      conn.setRequestMethod(operation.getHttpMethod());
+      HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+      if (hasResponse) {
+        String contentType = conn.getHeaderField(CONTENT_TYPE);
+        contentType =
+            (contentType != null) ? StringUtils.toLowerCase(contentType) : null;
+        if (contentType != null &&
+            contentType.contains(APPLICATION_JSON_MIME)) {
+          try {
+            ObjectMapper mapper = new ObjectMapper();
+            ret = mapper.readValue(conn.getInputStream(), Map.class);
+          } catch (Exception ex) {
+            throw new AuthenticationException(String.format(
+                "'%s' did not handle the '%s' delegation token operation: %s",
+                url.getAuthority(), operation, ex.getMessage()), ex);
+          }
+        } else {
+          throw new AuthenticationException(String.format("'%s' did not " +
+                  "respond with JSON to the '%s' delegation token operation",
+              url.getAuthority(), operation));
         }
-      } else {
-        throw new AuthenticationException(String.format("'%s' did not " +
-                "respond with JSON to the '%s' delegation token operation",
-            url.getAuthority(), operation));
+      }
+    } finally {
+      if (dt != null) {
+        ((DelegationTokenAuthenticatedURL.Token) token).setDelegationToken(dt);
       }
     }
     return ret;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771f798e/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenManager.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenManager.java
index 221b02a..b1a8d48 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenManager.java
@@ -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.AbstractDelegationTokenSecretManager;
 import org.apache.hadoop.security.token.delegation.ZKDelegationTokenSecretManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -41,6 +43,8 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class DelegationTokenManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DelegationTokenManager.class);
 
   public static final String ENABLE_ZK_KEY = "zk-dt-secret-manager.enable";
 
@@ -156,6 +160,7 @@ public class DelegationTokenManager {
   @SuppressWarnings("unchecked")
   public Token<? extends AbstractDelegationTokenIdentifier> createToken(
       UserGroupInformation ugi, String renewer) {
+    LOG.debug("Creating token with ugi:{}, renewer:{}.", ugi, renewer);
     renewer = (renewer == null) ? ugi.getShortUserName() : renewer;
     String user = ugi.getUserName();
     Text owner = new Text(user);
@@ -175,6 +180,7 @@ public class DelegationTokenManager {
   public long renewToken(
       Token<? extends AbstractDelegationTokenIdentifier> token, String renewer)
           throws IOException {
+    LOG.debug("Renewing token:{} with renewer:{}.", token, renewer);
     return secretManager.renewToken(token, renewer);
   }
 
@@ -182,6 +188,7 @@ public class DelegationTokenManager {
   public void cancelToken(
       Token<? extends AbstractDelegationTokenIdentifier> token,
       String canceler) throws IOException {
+    LOG.debug("Cancelling token:{} with canceler:{}.", token, canceler);
     canceler = (canceler != null) ? canceler :
                verifyToken(token).getShortUserName();
     secretManager.cancelToken(token, canceler);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771f798e/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
index 6866367..d50b0f4 100644
--- a/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
+++ b/hadoop-common-project/hadoop-kms/src/site/markdown/index.md.vm
@@ -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 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:
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/771f798e/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
index 94b9d06..e3b30a1 100644
--- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
+++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
@@ -35,13 +35,10 @@ 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;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -73,11 +70,6 @@ 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);
 
@@ -262,6 +254,7 @@ public class TestKMS {
       kdc = null;
     }
     UserGroupInformation.setShouldRenewImmediatelyForTests(false);
+    UserGroupInformation.reset();
   }
 
   private <T> T doAs(String user, final PrivilegedExceptionAction<T> action)
@@ -1748,93 +1741,136 @@ public class TestKMS {
   @Test
   public void testDelegationTokensOpsSimple() throws Exception {
     final Configuration conf = new Configuration();
-    final Authenticator mock = mock(PseudoAuthenticator.class);
-    testDelegationTokensOps(conf, mock);
+    testDelegationTokensOps(conf, false);
   }
 
   @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);
+    testDelegationTokensOps(conf, true);
   }
 
   private void testDelegationTokensOps(Configuration conf,
-      final Authenticator mockAuthenticator) throws Exception {
+      final boolean useKrb) throws Exception {
     UserGroupInformation.setConfiguration(conf);
     File confDir = getTestDir();
     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);
-    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,
+        final Configuration clientConf = new Configuration();
+        final URI uri = createKMSUri(getKMSUrl());
+        clientConf.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);
+        doAs("client", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            KeyProvider kp = createProvider(uri, clientConf);
+            // test delegation token retrieval
+            KeyProviderDelegationTokenExtension kpdte =
+                KeyProviderDelegationTokenExtension.
+                    createKeyProviderDelegationTokenExtension(kp);
+            final Credentials credentials = new Credentials();
+            final Token<?>[] tokens =
+                kpdte.addDelegationTokens("client1", 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());
+
+            // Test non-renewer user cannot renew.
+            for (Token<?> token : tokens) {
+              if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) {
+                LOG.info("Skipping token {}", token);
+                continue;
+              }
+              LOG.info("Got dt for " + uri + "; " + token);
+              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);
+              }
+            }
 
-        // 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);
+            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:{}",
+                        token.getKind(), tokenLife);
+                    Thread.sleep(100);
+                    long newTokenLife = token.renew(clientConf);
+                    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(clientConf);
+                    LOG.info("Cancelled token of kind {}", token.getKind());
+                    try {
+                      token.renew(clientConf);
+                      Assert
+                          .fail("should not be able to renew a canceled token");
+                    } catch (Exception e) {
+                      LOG.info("Expected exception when renewing token", e);
+                    }
+                  }
+                  return null;
+                }
+              });
+              return null;
+            } finally {
+              otherUgi.logoutUserFromKeytab();
+            }
           }
-        }
+        });
         return null;
       }
     });


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org