Browse Source

svn merge -c 1196812 from 0.23 for HADOOP-7792 and HDFS-2416.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23.0@1199413 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 years ago
parent
commit
425f76e4ce
15 changed files with 298 additions and 31 deletions
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 15 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
  3. 2 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java
  4. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  5. 12 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java
  6. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java
  7. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  8. 11 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  9. 22 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/AuthFilter.java
  10. 51 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java
  12. 44 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/TokenArgumentParam.java
  13. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java
  14. 22 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
  15. 93 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

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

@@ -441,6 +441,9 @@ Release 0.23.0 - 2011-11-01
 
     HADOOP-7789. Improvements to site navigation. (acmurthy) 
 
+    HADOOP-7792. Add verifyToken method to AbstractDelegationTokenSecretManager.
+    (jitendra)
+
   OPTIMIZATIONS
   
     HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole

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

@@ -209,6 +209,21 @@ extends AbstractDelegationTokenIdentifier>
     return info.getPassword();
   }
 
+  /**
+   * Verifies that the given identifier and password are valid and match.
+   * @param identifier Token identifier.
+   * @param password Password in the token.
+   * @throws InvalidToken
+   */
+  public synchronized void verifyToken(TokenIdent identifier, byte[] password)
+      throws InvalidToken {
+    byte[] storedPassword = retrievePassword(identifier);
+    if (!Arrays.equals(password, storedPassword)) {
+      throw new InvalidToken("token (" + identifier
+          + ") is invalid, password doesn't match");
+    }
+  }
+  
   /**
    * Renew a delegation token.
    * @param token the token to renew

+ 2 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java

@@ -360,6 +360,8 @@ public class TestDelegationToken {
         byte[] storedPassword = dtSecretManager.retrievePassword(id);
         byte[] password = dtSecretManager.createPassword(id, key);
         Assert.assertTrue(Arrays.equals(password, storedPassword));
+        //verify by secret manager api
+        dtSecretManager.verifyToken(id, password);
       }
     } finally {
       dtSecretManager.stopThreads();

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -1176,6 +1176,8 @@ Release 0.23.0 - 2011-11-01
 
     HDFS-2522. Disable TestDfsOverAvroRpc test. (suresh)
 
+    HDFS-2416. distcp with a webhdfs uri on a secure cluster fails. (jitendra)
+
   BREAKDOWN OF HDFS-1073 SUBTASKS
 
     HDFS-1521. Persist transaction ID on disk between NN restarts.

+ 12 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/ByteRangeInputStream.java

@@ -40,13 +40,13 @@ import org.apache.hadoop.hdfs.web.resources.OffsetParam;
 public class ByteRangeInputStream extends FSInputStream {
   
   /**
-   * This class wraps a URL to allow easy mocking when testing. The URL class
-   * cannot be easily mocked because it is public.
+   * This class wraps a URL and provides method to open connection.
+   * It can be overridden to change how a connection is opened.
    */
-  static class URLOpener {
+  public static class URLOpener {
     protected URL url;
     /** The url with offset parameter */
-    private URL offsetUrl;
+    protected URL offsetUrl;
   
     public URLOpener(URL u) {
       url = u;
@@ -60,7 +60,7 @@ public class ByteRangeInputStream extends FSInputStream {
       return url;
     }
 
-    HttpURLConnection openConnection() throws IOException {
+    protected HttpURLConnection openConnection() throws IOException {
       return (HttpURLConnection)offsetUrl.openConnection();
     }
 
@@ -125,7 +125,13 @@ public class ByteRangeInputStream extends FSInputStream {
     this(new URLOpener(url), new URLOpener(null));
   }
   
-  ByteRangeInputStream(URLOpener o, URLOpener r) {
+  /**
+   * Create with the specified URLOpeners. Original url is used to open the 
+   * stream for the first time. Resolved url is used in subsequent requests.
+   * @param o Original url
+   * @param r Resolved url
+   */
+  public ByteRangeInputStream(URLOpener o, URLOpener r) {
     this.originalURL = o;
     this.resolvedURL = r;
   }

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/JspHelper.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
@@ -552,6 +553,13 @@ public class JspHelper {
         DataInputStream in = new DataInputStream(buf);
         DelegationTokenIdentifier id = new DelegationTokenIdentifier();
         id.readFields(in);
+        if (context != null) {
+          final NameNode nn = NameNodeHttpServer.getNameNodeFromContext(context);
+          if (nn != null) {
+            // Verify the token.
+            nn.getNamesystem().verifyToken(id, token.getPassword());
+          }
+        }
         ugi = id.getUser();
         checkUsername(ugi.getShortUserName(), usernameFromQuery);
         checkUsername(ugi.getShortUserName(), user);

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -4371,4 +4371,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   public BlockManager getBlockManager() {
     return blockManager;
   }
+  
+  /**
+   * Verifies that the given identifier and password are valid and match.
+   * @param identifier Token identifier.
+   * @param password Password in the token.
+   * @throws InvalidToken
+   */
+  public synchronized void verifyToken(DelegationTokenIdentifier identifier,
+      byte[] password) throws InvalidToken {
+    getDelegationTokenSecretManager().verifyToken(identifier, password);
+  }
 }

+ 11 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
 import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
 import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
 import org.apache.hadoop.hdfs.web.resources.DelegationParam;
+import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
 import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
 import org.apache.hadoop.hdfs.web.resources.DestinationParam;
 import org.apache.hadoop.hdfs.web.resources.GetOpParam;
@@ -104,7 +105,7 @@ public class NamenodeWebHdfsMethods {
   public static final Log LOG = LogFactory.getLog(NamenodeWebHdfsMethods.class);
 
   private static final UriFsPathParam ROOT = new UriFsPathParam("");
-
+  
   private static final ThreadLocal<String> REMOTE_ADDRESS = new ThreadLocal<String>(); 
 
   /** @return the remote client address. */
@@ -224,11 +225,13 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(AccessTimeParam.NAME) @DefaultValue(AccessTimeParam.DEFAULT)
           final AccessTimeParam accessTime,
       @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
-          final RenameOptionSetParam renameOptions
+          final RenameOptionSetParam renameOptions,
+      @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT) 
+          final TokenArgumentParam delegationTokenArgument
       ) throws IOException, InterruptedException {
     return put(ugi, delegation, ROOT, op, destination, owner, group,
         permission, overwrite, bufferSize, replication, blockSize,
-        modificationTime, accessTime, renameOptions);
+        modificationTime, accessTime, renameOptions, delegationTokenArgument);
   }
 
   /** Handle HTTP PUT request. */
@@ -264,7 +267,9 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(AccessTimeParam.NAME) @DefaultValue(AccessTimeParam.DEFAULT)
           final AccessTimeParam accessTime,
       @QueryParam(RenameOptionSetParam.NAME) @DefaultValue(RenameOptionSetParam.DEFAULT)
-          final RenameOptionSetParam renameOptions
+          final RenameOptionSetParam renameOptions,
+      @QueryParam(TokenArgumentParam.NAME) @DefaultValue(TokenArgumentParam.DEFAULT) 
+          final TokenArgumentParam delegationTokenArgument
       ) throws IOException, InterruptedException {
 
     if (LOG.isTraceEnabled()) {
@@ -344,7 +349,7 @@ public class NamenodeWebHdfsMethods {
     case RENEWDELEGATIONTOKEN:
     {
       final Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>();
-      token.decodeFromUrlString(delegation.getValue());
+      token.decodeFromUrlString(delegationTokenArgument.getValue());
       final long expiryTime = np.renewDelegationToken(token);
       final String js = JsonUtil.toJsonString("long", expiryTime);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
@@ -352,7 +357,7 @@ public class NamenodeWebHdfsMethods {
     case CANCELDELEGATIONTOKEN:
     {
       final Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>();
-      token.decodeFromUrlString(delegation.getValue());
+      token.decodeFromUrlString(delegationTokenArgument.getValue());
       np.cancelDelegationToken(token);
       return Response.ok().type(MediaType.APPLICATION_JSON).build();
     }

+ 22 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/AuthFilter.java

@@ -17,11 +17,17 @@
  */
 package org.apache.hadoop.hdfs.web;
 
+import java.io.IOException;
 import java.util.Properties;
 
+import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
 
+import org.apache.hadoop.hdfs.web.resources.DelegationParam;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
@@ -55,6 +61,21 @@ public class AuthFilter extends AuthenticationFilter {
     p.setProperty(PseudoAuthenticationHandler.ANONYMOUS_ALLOWED, "true");
     //set cookie path
     p.setProperty(COOKIE_PATH, "/");
-   return p;
+    return p;
+  }
+
+  @Override
+  public void doFilter(ServletRequest request, ServletResponse response,
+      FilterChain filterChain) throws IOException, ServletException {
+    HttpServletRequest httpRequest = (HttpServletRequest) request;
+    String tokenString = httpRequest
+        .getParameter(DelegationParam.NAME);
+    if (tokenString != null) {
+      //Token is present in the url, therefore token will be used for
+      //authentication, bypass kerberos authentication.
+      filterChain.doFilter(httpRequest, response);
+      return;
+    }
+    super.doFilter(request, response, filterChain);
   }
 }

+ 51 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
 import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
 import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
+import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
 import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
 import org.apache.hadoop.hdfs.web.resources.DestinationParam;
 import org.apache.hadoop.hdfs.web.resources.GetOpParam;
@@ -290,24 +291,42 @@ public class WebHdfsFileSystem extends FileSystem
     final String query = op.toQueryString()
         + '&' + new UserParam(ugi)
         + Param.toSortedString("&", parameters);
-    final URL url = getNamenodeURL(path, addDt2Query(query));
+    final URL url;
+    if (op.equals(PutOpParam.Op.RENEWDELEGATIONTOKEN)
+        || op.equals(GetOpParam.Op.GETDELEGATIONTOKEN)) {
+      // Skip adding delegation token for getting or renewing delegation token,
+      // because these operations require kerberos authentication.
+      url = getNamenodeURL(path, query);
+    } else {
+      url = getNamenodeURL(path, addDt2Query(query));
+    }
     if (LOG.isTraceEnabled()) {
       LOG.trace("url=" + url);
     }
     return url;
   }
 
+  private HttpURLConnection getHttpUrlConnection(URL url)
+      throws IOException {
+    final HttpURLConnection conn;
+    try {
+      if (ugi.hasKerberosCredentials()) { 
+        conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
+      } else {
+        conn = (HttpURLConnection)url.openConnection();
+      }
+    } catch (AuthenticationException e) {
+      throw new IOException("Authentication failed, url=" + url, e);
+    }
+    return conn;
+  }
+  
   private HttpURLConnection httpConnect(final HttpOpParam.Op op, final Path fspath,
       final Param<?,?>... parameters) throws IOException {
     final URL url = toUrl(op, fspath, parameters);
 
     //connect and get response
-    final HttpURLConnection conn;
-    try {
-      conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
-    } catch(AuthenticationException e) {
-      throw new IOException("Authentication failed, url=" + url, e);
-    }
+    final HttpURLConnection conn = getHttpUrlConnection(url);
     try {
       conn.setRequestMethod(op.getType().toString());
       conn.setDoOutput(op.getDoOutput());
@@ -317,7 +336,7 @@ public class WebHdfsFileSystem extends FileSystem
       }
       conn.connect();
       return conn;
-    } catch(IOException e) {
+    } catch (IOException e) {
       conn.disconnect();
       throw e;
     }
@@ -513,7 +532,24 @@ public class WebHdfsFileSystem extends FileSystem
     statistics.incrementReadOps(1);
     final HttpOpParam.Op op = GetOpParam.Op.OPEN;
     final URL url = toUrl(op, f, new BufferSizeParam(buffersize));
-    return new FSDataInputStream(new ByteRangeInputStream(url));
+    ByteRangeInputStream str = getByteRangeInputStream(url);
+    return new FSDataInputStream(str);
+  }
+
+  private class URLOpener extends ByteRangeInputStream.URLOpener {
+
+    public URLOpener(URL u) {
+      super(u);
+    }
+
+    @Override
+    public HttpURLConnection openConnection() throws IOException {
+      return getHttpUrlConnection(offsetUrl);
+    }
+  }
+  
+  private ByteRangeInputStream getByteRangeInputStream(URL url) {
+    return new ByteRangeInputStream(new URLOpener(url), new URLOpener(null));
   }
 
   @Override
@@ -576,17 +612,19 @@ public class WebHdfsFileSystem extends FileSystem
 
   private synchronized long renewDelegationToken(final Token<?> token
       ) throws IOException {
-    delegationToken = token;
     final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
-    final Map<?, ?> m = run(op, null);
+    TokenArgumentParam dtargParam = new TokenArgumentParam(
+        token.encodeToUrlString());
+    final Map<?, ?> m = run(op, null, dtargParam);
     return (Long) m.get("long");
   }
 
   private synchronized void cancelDelegationToken(final Token<?> token
       ) throws IOException {
-    delegationToken = token;
     final HttpOpParam.Op op = PutOpParam.Op.CANCELDELEGATIONTOKEN;
-    run(op, null);
+    TokenArgumentParam dtargParam = new TokenArgumentParam(
+        token.encodeToUrlString());
+    run(op, null, dtargParam);
   }
   
   @Override

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DelegationParam.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.web.resources;
 
 import org.apache.hadoop.security.UserGroupInformation;
 
-/** Delegation token parameter. */
+/** Represents delegation token used for authentication. */
 public class DelegationParam extends StringParam {
   /** Parameter name. */
   public static final String NAME = "delegation";

+ 44 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/TokenArgumentParam.java

@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/**
+ * Represents delegation token parameter as method arguments. This is
+ * different from {@link DelegationParam}.
+ */
+public class TokenArgumentParam extends StringParam {
+  /** Parameter name. */
+  public static final String NAME = "token";
+  /** Default parameter value. */
+  public static final String DEFAULT = "";
+
+  private static final Domain DOMAIN = new Domain(NAME, null);
+
+  /**
+   * Constructor.
+   * @param str A string representation of the parameter value.
+   */
+  public TokenArgumentParam(final String str) {
+    super(DOMAIN, str != null && !str.equals(DEFAULT) ? str : null);
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/UserProvider.java

@@ -50,7 +50,7 @@ public class UserProvider
     final Configuration conf = (Configuration) servletcontext
         .getAttribute(JspHelper.CURRENT_CONF);
     try {
-      return JspHelper.getUGI(null, request, conf,
+      return JspHelper.getUGI(servletcontext, request, conf,
           AuthenticationMethod.KERBEROS, false);
     } catch (IOException e) {
       throw new RuntimeException(e);

+ 22 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java

@@ -148,7 +148,7 @@ public class TestDelegationToken {
   @Test
   public void testDelegationTokenDFSApi() throws Exception {
     DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
-    Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
+    final Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
     DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
     byte[] tokenId = token.getIdentifier();
     identifier.readFields(new DataInputStream(
@@ -156,6 +156,15 @@ public class TestDelegationToken {
     LOG.info("A valid token should have non-null password, and should be renewed successfully");
     Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
     dtSecretManager.renewToken(token, "JobTracker");
+    UserGroupInformation.createRemoteUser("JobTracker").doAs(
+        new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            token.renew(config);
+            token.cancel(config);
+            return null;
+          }
+        });
   }
   
   @SuppressWarnings("deprecation")
@@ -175,13 +184,23 @@ public class TestDelegationToken {
       }
     });
 
-    final Token<DelegationTokenIdentifier> token = webhdfs.getDelegationToken("JobTracker");
+    final Token<DelegationTokenIdentifier> token = webhdfs
+        .getDelegationToken("JobTracker");
     DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
     byte[] tokenId = token.getIdentifier();
-    identifier.readFields(new DataInputStream(new ByteArrayInputStream(tokenId)));
+    identifier
+        .readFields(new DataInputStream(new ByteArrayInputStream(tokenId)));
     LOG.info("A valid token should have non-null password, and should be renewed successfully");
     Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
     dtSecretManager.renewToken(token, "JobTracker");
+    ugi.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        token.renew(config);
+        token.cancel(config);
+        return null;
+      }
+    });
   }
 
   @Test

+ 93 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.web;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URL;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.web.resources.DelegationParam;
+import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.junit.Assert;
+import org.junit.Test;
+import static org.mockito.Mockito.mock;
+
+public class TestWebHdfsUrl {
+
+  @Test
+  public void testDelegationTokenInUrl() throws IOException {
+    Configuration conf = new Configuration();
+    final String uri = WebHdfsFileSystem.SCHEME + "://" + "127.0.0.1:9071";
+    // Turn on security
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+    DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(new Text(
+        ugi.getUserName()), null, null);
+    FSNamesystem namesystem = mock(FSNamesystem.class);
+    DelegationTokenSecretManager dtSecretManager = new DelegationTokenSecretManager(
+        86400000, 86400000, 86400000, 86400000, namesystem);
+    dtSecretManager.startThreads();
+    Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>(
+        dtId, dtSecretManager);
+    token.setService(new Text("127.0.0.1:9071"));
+    token.setKind(WebHdfsFileSystem.TOKEN_KIND);
+    ugi.addToken(token);
+    final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem) FileSystem.get(
+        URI.create(uri), conf);
+    String tokenString = token.encodeToUrlString();
+    Path fsPath = new Path("/");
+    URL renewTokenUrl = webhdfs.toUrl(PutOpParam.Op.RENEWDELEGATIONTOKEN,
+        fsPath, new TokenArgumentParam(tokenString));
+    URL cancelTokenUrl = webhdfs.toUrl(PutOpParam.Op.CANCELDELEGATIONTOKEN,
+        fsPath, new TokenArgumentParam(tokenString));
+    Assert.assertEquals(
+        generateUrlQueryPrefix(PutOpParam.Op.RENEWDELEGATIONTOKEN,
+            ugi.getUserName())
+            + "&token=" + tokenString, renewTokenUrl.getQuery());
+    Token<DelegationTokenIdentifier> delegationToken = new Token<DelegationTokenIdentifier>(
+        token);
+    delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
+    Assert.assertEquals(
+        generateUrlQueryPrefix(PutOpParam.Op.CANCELDELEGATIONTOKEN,
+            ugi.getUserName())
+            + "&token="
+            + tokenString
+            + "&"
+            + DelegationParam.NAME
+            + "="
+            + delegationToken.encodeToUrlString(), cancelTokenUrl.getQuery());
+  }
+
+  private String generateUrlQueryPrefix(HttpOpParam.Op op, String username) {
+    return "op=" + op.toString() + "&user.name=" + username;
+  }
+}