Sfoglia il codice sorgente

HADOOP-13174. Add more debug logs for delegation tokens and authentication.

(cherry picked from commit 4a56bde6ba1f72588a25cd96acc76089706cb786)

Conflicts:
	hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java
(cherry picked from commit ffaf24e308506ec4c27104bf6b3769328e55c1c6)

 Conflicts:
	hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java
	hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
Xiao Chen 8 anni fa
parent
commit
5f8ab3a6b7

+ 1 - 0
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java

@@ -388,6 +388,7 @@ public class AuthenticatedURL {
       // not opened via this instance.
       token.cookieHandler.put(null, conn.getHeaderFields());
     } else {
+      LOG.trace("Setting token value to null ({}), resp={}", token, respCode);
       token.set(null);
       throw new AuthenticationException("Authentication failed, status: " + conn.getResponseCode() +
                                         ", message: " + conn.getResponseMessage());

+ 10 - 2
hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java

@@ -516,6 +516,10 @@ public class AuthenticationFilter implements Filter {
       AuthenticationToken token;
       try {
         token = getToken(httpRequest);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Got token {} from httpRequest {}", token,
+              getRequestURL(httpRequest));
+        }
       }
       catch (AuthenticationException ex) {
         LOG.warn("AuthenticationToken ignored: " + ex.getMessage());
@@ -526,8 +530,8 @@ public class AuthenticationFilter implements Filter {
       if (authHandler.managementOperation(token, httpRequest, httpResponse)) {
         if (token == null) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Request [{}] triggering authentication",
-                getRequestURL(httpRequest));
+            LOG.debug("Request [{}] triggering authentication. handler: {}",
+                getRequestURL(httpRequest), authHandler.getClass());
           }
           token = authHandler.authenticate(httpRequest, httpResponse);
           if (token != null && token != AuthenticationToken.ANONYMOUS) {
@@ -588,6 +592,10 @@ public class AuthenticationFilter implements Filter {
           doFilter(filterChain, httpRequest, httpResponse);
         }
       } else {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("managementOperation returned false for request {}."
+                  + " token: {}", getRequestURL(httpRequest), token);
+        }
         unauthorizedResponse = false;
       }
     } catch (AuthenticationException ex) {

+ 1 - 0
hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java

@@ -1232,6 +1232,7 @@ public class TestAuthenticationFilter {
       String tokenSigned = signer.sign(token.toString());
       Cookie cookie = new Cookie(AuthenticatedURL.AUTH_COOKIE, tokenSigned);
       Mockito.when(request.getCookies()).thenReturn(new Cookie[]{cookie});
+      Mockito.when(request.getRequestURL()).thenReturn(new StringBuffer());
 
       filter.doFilter(request, response, chain);
 

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java

@@ -593,6 +593,10 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
       // failure. Unfortunately, the AuthenticationFilter returns 403 when it
       // cannot authenticate (Since a 401 requires Server to send
       // WWW-Authenticate header as well)..
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Response={}({}), resetting authToken",
+            conn.getResponseCode(), conn.getResponseMessage());
+      }
       KMSClientProvider.this.authToken =
           new DelegationTokenAuthenticatedURL.Token();
       if (authRetryCount > 0) {
@@ -1010,11 +1014,13 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
           public Token<?> run() throws Exception {
             // Not using the cached token here.. Creating a new token here
             // everytime.
+            LOG.debug("Getting new token from {}, renewer:{}", url, renewer);
             return authUrl.getDelegationToken(url,
                 new DelegationTokenAuthenticatedURL.Token(), renewer, doAsUser);
           }
         });
         if (token != null) {
+          LOG.debug("New token received: ({})", token);
           credentials.addToken(token.getService(), token);
           tokens = new Token<?>[] { token };
         } else {

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java

@@ -635,6 +635,7 @@ extends AbstractDelegationTokenIdentifier>
       Collection<TokenIdent> expiredTokens) throws IOException {
     for (TokenIdent ident : expiredTokens) {
       logExpireToken(ident);
+      LOG.info("Removing expired token " + formatTokenId(ident));
       removeStoredToken(ident);
     }
   }

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java

@@ -29,6 +29,8 @@ import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.net.HttpURLConnection;
@@ -61,6 +63,9 @@ import java.util.Map;
 @InterfaceStability.Unstable
 public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DelegationTokenAuthenticatedURL.class);
+
   /**
    * Constant used in URL's query string to perform a proxy user request, the
    * value of the <code>DO_AS</code> parameter is the user the request will be
@@ -283,17 +288,23 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
     Map<String, String> extraParams = new HashMap<String, String>();
     org.apache.hadoop.security.token.Token<? extends TokenIdentifier> dToken
         = null;
+    LOG.debug("Connecting to url {} with token {} as {}", url, token, doAs);
     // if we have valid auth token, it takes precedence over a delegation token
     // and we don't even look for one.
     if (!token.isSet()) {
       // delegation token
       Credentials creds = UserGroupInformation.getCurrentUser().
           getCredentials();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Token not set, looking for delegation token. Creds:{}",
+            creds.getAllTokens());
+      }
       if (!creds.getAllTokens().isEmpty()) {
         InetSocketAddress serviceAddr = new InetSocketAddress(url.getHost(),
             url.getPort());
         Text service = SecurityUtil.buildTokenService(serviceAddr);
         dToken = creds.getToken(service);
+        LOG.debug("Using delegation token {} from service:{}", dToken, service);
         if (dToken != null) {
           if (useQueryStringForDelegationToken()) {
             // delegation token will go in the query string, injecting it

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java

@@ -49,6 +49,8 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecret
 import org.apache.hadoop.util.HttpExceptionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -77,6 +79,8 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceStability.Evolving
 public abstract class DelegationTokenAuthenticationHandler
     implements AuthenticationHandler {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DelegationTokenAuthenticationHandler.class);
 
   protected static final String TYPE_POSTFIX = "-dt";
 
@@ -190,6 +194,7 @@ public abstract class DelegationTokenAuthenticationHandler
       HttpServletRequest request, HttpServletResponse response)
       throws IOException, AuthenticationException {
     boolean requestContinues = true;
+    LOG.trace("Processing operation for req=({}), token: {}", request, token);
     String op = ServletUtils.getParameter(request,
         KerberosDelegationTokenAuthenticator.OP_PARAM);
     op = (op != null) ? StringUtils.toUpperCase(op) : null;
@@ -202,6 +207,7 @@ public abstract class DelegationTokenAuthenticationHandler
         if (dtOp.requiresKerberosCredentials() && token == null) {
           // Don't authenticate via DT for DT ops.
           token = authHandler.authenticate(request, response);
+          LOG.trace("Got token: {}.", token);
           if (token == null) {
             requestContinues = false;
             doManagement = false;
@@ -353,6 +359,7 @@ public abstract class DelegationTokenAuthenticationHandler
     AuthenticationToken token;
     String delegationParam = getDelegationToken(request);
     if (delegationParam != null) {
+      LOG.debug("Authenticating with dt param: {}", delegationParam);
       try {
         Token<AbstractDelegationTokenIdentifier> dt = new Token();
         dt.decodeFromUrlString(delegationParam);
@@ -370,6 +377,7 @@ public abstract class DelegationTokenAuthenticationHandler
             HttpServletResponse.SC_FORBIDDEN, new AuthenticationException(ex));
       }
     } else {
+      LOG.debug("Falling back to {} (req={})", authHandler.getClass(), request);
       token = authHandler.authenticate(request, response);
     }
     return token;

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java

@@ -116,10 +116,16 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     if (token instanceof DelegationTokenAuthenticatedURL.Token) {
       hasDt = ((DelegationTokenAuthenticatedURL.Token) token).
           getDelegationToken() != null;
+      if (hasDt) {
+        LOG.trace("Delegation token found: {}",
+            ((DelegationTokenAuthenticatedURL.Token) token)
+                .getDelegationToken());
+      }
     }
     if (!hasDt) {
       String queryStr = url.getQuery();
       hasDt = (queryStr != null) && queryStr.contains(DELEGATION_PARAM + "=");
+      LOG.trace("hasDt={}, queryStr={}", hasDt, queryStr);
     }
     return hasDt;
   }
@@ -130,7 +136,12 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     if (!hasDelegationToken(url, token)) {
       // check and renew TGT to handle potential expiration
       UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
+      LOG.debug("No delegation token found for url={}, token={}, authenticating"
+          + " with {}", url, token, authenticator.getClass());
       authenticator.authenticate(url, token);
+    } else {
+      LOG.debug("Authenticated from delegation token. url={}, token={}",
+          url, token);
     }
   }