Browse Source

HADOOP-10880. Move HTTP delegation tokens out of URL querystring to a header. (tucu)

Alejandro Abdelnur 10 năm trước cách đây
mục cha
commit
d1ae479aa5

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -518,6 +518,9 @@ Release 2.6.0 - UNRELEASED
 
     HADOOP-10998. Fix bash tab completion code to work (Jim Hester via aw)
 
+    HADOOP-10880. Move HTTP delegation tokens out of URL querystring to 
+    a header. (tucu)
+
   OPTIMIZATIONS
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)

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

@@ -125,6 +125,8 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
     }
   }
 
+  private boolean useQueryStringforDelegationToken = false;
+
   /**
    * Creates an <code>DelegationTokenAuthenticatedURL</code>.
    * <p/>
@@ -170,6 +172,34 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
     super(obtainDelegationTokenAuthenticator(authenticator), connConfigurator);
   }
 
+  /**
+   * Sets if delegation token should be transmitted in the URL query string.
+   * By default it is transmitted using the
+   * {@link DelegationTokenAuthenticator#DELEGATION_TOKEN_HEADER} HTTP header.
+   * <p/>
+   * This method is provided to enable WebHDFS backwards compatibility.
+   *
+   * @param useQueryString  <code>TRUE</code> if the token is transmitted in the
+   * URL query string, <code>FALSE</code> if the delegation token is transmitted
+   * using the {@link DelegationTokenAuthenticator#DELEGATION_TOKEN_HEADER} HTTP
+   * header.
+   */
+  @Deprecated
+  protected void setUseQueryStringForDelegationToken(boolean useQueryString) {
+    useQueryStringforDelegationToken = useQueryString;
+  }
+
+  /**
+   * Returns if delegation token is transmitted as a HTTP header.
+   *
+   * @return <code>TRUE</code> if the token is transmitted in the URL query
+   * string, <code>FALSE</code> if the delegation token is transmitted using the
+   * {@link DelegationTokenAuthenticator#DELEGATION_TOKEN_HEADER} HTTP header.
+   */
+  public boolean useQueryStringForDelegationToken() {
+    return useQueryStringforDelegationToken;
+  }
+
   /**
    * Returns an authenticated {@link HttpURLConnection}, it uses a Delegation
    * Token only if the given auth token is an instance of {@link Token} and
@@ -235,23 +265,41 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
    * @throws IOException if an IO error occurred.
    * @throws AuthenticationException if an authentication exception occurred.
    */
+  @SuppressWarnings("unchecked")
   public HttpURLConnection openConnection(URL url, Token token, String doAs)
       throws IOException, AuthenticationException {
     Preconditions.checkNotNull(url, "url");
     Preconditions.checkNotNull(token, "token");
     Map<String, String> extraParams = new HashMap<String, String>();
-
-    // delegation token
-    Credentials creds = UserGroupInformation.getCurrentUser().getCredentials();
-    if (!creds.getAllTokens().isEmpty()) {
-      InetSocketAddress serviceAddr = new InetSocketAddress(url.getHost(),
-          url.getPort());
-      Text service = SecurityUtil.buildTokenService(serviceAddr);
-      org.apache.hadoop.security.token.Token<? extends TokenIdentifier> dt =
-          creds.getToken(service);
-      if (dt != null) {
-        extraParams.put(KerberosDelegationTokenAuthenticator.DELEGATION_PARAM,
-            dt.encodeToUrlString());
+    org.apache.hadoop.security.token.Token<? extends TokenIdentifier> dToken
+        = null;
+    // 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 (!creds.getAllTokens().isEmpty()) {
+        InetSocketAddress serviceAddr = new InetSocketAddress(url.getHost(),
+            url.getPort());
+        Text service = SecurityUtil.buildTokenService(serviceAddr);
+        dToken = creds.getToken(service);
+        if (dToken != null) {
+          if (useQueryStringForDelegationToken()) {
+            // delegation token will go in the query string, injecting it
+            extraParams.put(
+                KerberosDelegationTokenAuthenticator.DELEGATION_PARAM,
+                dToken.encodeToUrlString());
+          } else {
+            // delegation token will go as request header, setting it in the
+            // auth-token to ensure no authentication handshake is triggered
+            // (if we have a delegation token, we are authenticated)
+            // the delegation token header is injected in the connection request
+            // at the end of this method.
+            token.delegationToken = (org.apache.hadoop.security.token.Token
+                <AbstractDelegationTokenIdentifier>) dToken;
+          }
+        }
       }
     }
 
@@ -261,7 +309,14 @@ public class DelegationTokenAuthenticatedURL extends AuthenticatedURL {
     }
 
     url = augmentURL(url, extraParams);
-    return super.openConnection(url, token);
+    HttpURLConnection conn = super.openConnection(url, token);
+    if (!token.isSet() && !useQueryStringForDelegationToken() && dToken != null) {
+      // injecting the delegation token header in the connection request
+      conn.setRequestProperty(
+          DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER,
+          dToken.encodeToUrlString());
+    }
+    return conn;
   }
 
   /**

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

@@ -331,8 +331,7 @@ public abstract class DelegationTokenAuthenticationHandler
       HttpServletResponse response)
       throws IOException, AuthenticationException {
     AuthenticationToken token;
-    String delegationParam = ServletUtils.getParameter(request,
-        KerberosDelegationTokenAuthenticator.DELEGATION_PARAM);
+    String delegationParam = getDelegationToken(request);
     if (delegationParam != null) {
       try {
         Token<DelegationTokenIdentifier> dt =
@@ -356,4 +355,15 @@ public abstract class DelegationTokenAuthenticationHandler
     return token;
   }
 
+  private String getDelegationToken(HttpServletRequest request)
+      throws IOException {
+    String dToken = request.getHeader(
+        DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER);
+    if (dToken == null) {
+      dToken = ServletUtils.getParameter(request,
+          KerberosDelegationTokenAuthenticator.DELEGATION_PARAM);
+    }
+    return dToken;
+  }
+
 }

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

@@ -56,6 +56,9 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
 
   public static final String OP_PARAM = "op";
 
+  public static final String DELEGATION_TOKEN_HEADER =
+      "X-Hadoop-Delegation-Token";
+
   public static final String DELEGATION_PARAM = "delegation";
   public static final String TOKEN_PARAM = "token";
   public static final String RENEWER_PARAM = "renewer";
@@ -101,15 +104,23 @@ public abstract class DelegationTokenAuthenticator implements Authenticator {
     authenticator.setConnectionConfigurator(configurator);
   }
 
-  private boolean hasDelegationToken(URL url) {
-    String queryStr = url.getQuery();
-    return (queryStr != null) && queryStr.contains(DELEGATION_PARAM + "=");
+  private boolean hasDelegationToken(URL url, AuthenticatedURL.Token token) {
+    boolean hasDt = false;
+    if (token instanceof DelegationTokenAuthenticatedURL.Token) {
+      hasDt = ((DelegationTokenAuthenticatedURL.Token) token).
+          getDelegationToken() != null;
+    }
+    if (!hasDt) {
+      String queryStr = url.getQuery();
+      hasDt = (queryStr != null) && queryStr.contains(DELEGATION_PARAM + "=");
+    }
+    return hasDt;
   }
 
   @Override
   public void authenticate(URL url, AuthenticatedURL.Token token)
       throws IOException, AuthenticationException {
-    if (!hasDelegationToken(url)) {
+    if (!hasDelegationToken(url, token)) {
       authenticator.authenticate(url, token);
     }
   }

+ 42 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestDelegationTokenAuthenticationHandlerWithMocks.java

@@ -284,11 +284,13 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
 
   @Test
   public void testAuthenticate() throws Exception {
-    testValidDelegationToken();
-    testInvalidDelegationToken();
+    testValidDelegationTokenQueryString();
+    testValidDelegationTokenHeader();
+    testInvalidDelegationTokenQueryString();
+    testInvalidDelegationTokenHeader();
   }
 
-  private void testValidDelegationToken() throws Exception {
+  private void testValidDelegationTokenQueryString() throws Exception {
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
     Token<DelegationTokenIdentifier> dToken =
@@ -307,7 +309,26 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     Assert.assertTrue(token.isExpired());
   }
 
-  private void testInvalidDelegationToken() throws Exception {
+  private void testValidDelegationTokenHeader() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Token<DelegationTokenIdentifier> dToken =
+        handler.getTokenManager().createToken(
+            UserGroupInformation.getCurrentUser(), "user");
+    Mockito.when(request.getHeader(Mockito.eq(
+        DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER))).thenReturn(
+        dToken.encodeToUrlString());
+
+    AuthenticationToken token = handler.authenticate(request, response);
+    Assert.assertEquals(UserGroupInformation.getCurrentUser().
+        getShortUserName(), token.getUserName());
+    Assert.assertEquals(0, token.getExpires());
+    Assert.assertEquals(handler.getType(),
+        token.getType());
+    Assert.assertTrue(token.isExpired());
+  }
+
+  private void testInvalidDelegationTokenQueryString() throws Exception {
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
     Mockito.when(request.getQueryString()).thenReturn(
@@ -323,4 +344,21 @@ public class TestDelegationTokenAuthenticationHandlerWithMocks {
     }
   }
 
+  private void testInvalidDelegationTokenHeader() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(request.getHeader(Mockito.eq(
+        DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER))).thenReturn(
+        "invalid");
+
+    try {
+      handler.authenticate(request, response);
+      Assert.fail();
+    } catch (AuthenticationException ex) {
+      //NOP
+    } catch (Exception ex) {
+      Assert.fail();
+    }
+  }
+
 }

+ 47 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/web/TestWebDelegationToken.java

@@ -149,6 +149,15 @@ public class TestWebDelegationToken {
         throws ServletException, IOException {
       resp.setStatus(HttpServletResponse.SC_OK);
       resp.getWriter().write("ping");
+      if (req.getHeader(DelegationTokenAuthenticator.DELEGATION_TOKEN_HEADER)
+          != null) {
+        resp.setHeader("UsingHeader", "true");
+      }
+      if (req.getQueryString() != null &&
+          req.getQueryString().contains(
+              DelegationTokenAuthenticator.DELEGATION_PARAM + "=")) {
+        resp.setHeader("UsingQueryString", "true");
+      }
     }
 
     @Override
@@ -314,7 +323,20 @@ public class TestWebDelegationToken {
   }
 
   @Test
-  public void testDelegationTokenAuthenticatorCalls() throws Exception {
+  public void testDelegationTokenAuthenticatorCallsWithHeader()
+      throws Exception {
+    testDelegationTokenAuthenticatorCalls(false);
+  }
+
+  @Test
+  public void testDelegationTokenAuthenticatorCallsWithQueryString()
+      throws Exception {
+    testDelegationTokenAuthenticatorCalls(true);
+  }
+
+
+  private void testDelegationTokenAuthenticatorCalls(final boolean useQS)
+      throws Exception {
     final Server jetty = createJettyServer();
     Context context = new Context();
     context.setContextPath("/foo");
@@ -324,14 +346,15 @@ public class TestWebDelegationToken {
 
     try {
       jetty.start();
-      URL nonAuthURL = new URL(getJettyURL() + "/foo/bar");
+      final URL nonAuthURL = new URL(getJettyURL() + "/foo/bar");
       URL authURL = new URL(getJettyURL() + "/foo/bar?authenticated=foo");
       URL authURL2 = new URL(getJettyURL() + "/foo/bar?authenticated=bar");
 
       DelegationTokenAuthenticatedURL.Token token =
           new DelegationTokenAuthenticatedURL.Token();
-      DelegationTokenAuthenticatedURL aUrl =
+      final DelegationTokenAuthenticatedURL aUrl =
           new DelegationTokenAuthenticatedURL();
+      aUrl.setUseQueryStringForDelegationToken(useQS);
 
       try {
         aUrl.getDelegationToken(nonAuthURL, token, FOO_USER);
@@ -379,6 +402,27 @@ public class TestWebDelegationToken {
         Assert.assertTrue(ex.getMessage().contains("401"));
       }
 
+      aUrl.getDelegationToken(authURL, token, "foo");
+
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      ugi.addToken(token.getDelegationToken());
+      ugi.doAs(new PrivilegedExceptionAction<Void>() {
+                 @Override
+                 public Void run() throws Exception {
+                   HttpURLConnection conn = aUrl.openConnection(nonAuthURL, new DelegationTokenAuthenticatedURL.Token());
+                   Assert.assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+                   if (useQS) {
+                     Assert.assertNull(conn.getHeaderField("UsingHeader"));
+                     Assert.assertNotNull(conn.getHeaderField("UsingQueryString"));
+                   } else {
+                     Assert.assertNotNull(conn.getHeaderField("UsingHeader"));
+                     Assert.assertNull(conn.getHeaderField("UsingQueryString"));
+                   }
+                   return null;
+                 }
+               });
+
+
     } finally {
       jetty.stop();
     }