Forráskód Böngészése

Merge -r 1365987:1365988 from trunk to branch. FIXES: HDFS-3113

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1365989 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 12 éve
szülő
commit
c7bbda5ec9
32 módosított fájl, 3251 hozzáadás és 193 törlés
  1. 5 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/dev-support/findbugsExcludeFile.xml
  2. 58 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
  3. 158 125
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  4. 226 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSPseudoAuthenticator.java
  6. 148 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
  7. 6 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java
  8. 255 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandler.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java
  10. 5 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java
  11. 57 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenIdentifier.java
  12. 76 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenManager.java
  13. 49 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenManagerException.java
  14. 231 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/security/DelegationTokenManagerService.java
  15. 65 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java
  16. 29 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml
  17. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/webapp/WEB-INF/web.xml
  18. 14 4
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java
  19. 513 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java.orig
  20. 2 14
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestWebhdfsFileSystem.java
  21. 11 17
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandlerForTesting.java
  22. 310 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java
  23. 111 18
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
  24. 236 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java.orig
  25. 291 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java
  26. 83 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestDelegationTokenManagerService.java
  27. 138 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/KerberosTestUtils.java
  28. 1 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestDirHelper.java
  29. 22 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java
  30. 118 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java.orig
  31. 28 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/resources/krb5.conf
  32. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/dev-support/findbugsExcludeFile.xml

@@ -25,4 +25,9 @@
     <Method name="destroy" />
     <Bug pattern="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD" />
   </Match>
+  <Match>
+    <Class name="org.apache.hadoop.lib.servlet.ServerWebApp" />
+    <Field name="authority" />
+    <Bug pattern="IS2_INCONSISTENT_SYNC" />
+  </Match>
 </FindBugsFilter>

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml

@@ -43,6 +43,8 @@
     <httpfs.tomcat.dist.dir>
       ${project.build.directory}/${project.artifactId}-${project.version}/share/hadoop/httpfs/tomcat
     </httpfs.tomcat.dist.dir>
+    <kerberos.realm>LOCALHOST</kerberos.realm>
+    <test.exclude.kerberos.test>**/TestHttpFSWithKerberos.java</test.exclude.kerberos.test>
   </properties>
 
   <dependencies>
@@ -267,6 +269,22 @@
         </excludes>
       </resource>
     </resources>
+    <testResources>
+      <testResource>
+        <directory>${basedir}/src/test/resources</directory>
+        <filtering>false</filtering>
+        <excludes>
+          <exclude>krb5.conf</exclude>
+        </excludes>
+      </testResource>
+      <testResource>
+        <directory>${basedir}/src/test/resources</directory>
+        <filtering>true</filtering>
+        <includes>
+          <include>krb5.conf</include>
+        </includes>
+      </testResource>
+    </testResources>
 
     <plugins>
       <plugin>
@@ -281,6 +299,16 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <threadCount>1</threadCount>
+          <forkedProcessTimeoutInSeconds>600</forkedProcessTimeoutInSeconds>
+          <systemPropertyVariables>
+            <java.security.krb5.conf>${project.build.directory}/test-classes/krb5.conf</java.security.krb5.conf>
+            <kerberos.realm>${kerberos.realm}</kerberos.realm>
+          </systemPropertyVariables>
+          <excludes>
+            <exclude>**/${test.exclude}.java</exclude>
+            <exclude>${test.exclude.pattern}</exclude>
+            <exclude>${test.exclude.kerberos.test}</exclude>
+          </excludes>
         </configuration>
       </plugin>
       <plugin>
@@ -395,6 +423,36 @@
   </build>
 
   <profiles>
+    <profile>
+      <id>testKerberos</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <properties>
+        <test.exclude.kerberos.test>_</test.exclude.kerberos.test>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <configuration>
+              <forkMode>once</forkMode>
+              <forkedProcessTimeoutInSeconds>600</forkedProcessTimeoutInSeconds>
+              <systemPropertyVariables>
+                <java.security.krb5.conf>${project.build.directory}/test-classes/krb5.conf</java.security.krb5.conf>
+                <kerberos.realm>${kerberos.realm}</kerberos.realm>
+                <httpfs.http.hostname>localhost</httpfs.http.hostname>
+              </systemPropertyVariables>
+              <includes>
+                <include>**/TestHttpFSWithKerberos.java</include>
+              </includes>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    
     <profile>
       <id>docs</id>
       <activation>

+ 158 - 125
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.fs.http.client;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.DelegationTokenRenewer;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
@@ -28,16 +29,18 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PositionedReadable;
 import org.apache.hadoop.fs.Seekable;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.net.NetUtils;
 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.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
-import org.json.simple.parser.JSONParser;
-import org.json.simple.parser.ParseException;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -47,30 +50,32 @@ import java.io.FileNotFoundException;
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InputStreamReader;
 import java.io.OutputStream;
-import java.lang.reflect.Constructor;
 import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
-import java.net.URLEncoder;
+import java.security.PrivilegedExceptionAction;
 import java.text.MessageFormat;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
 
 /**
  * HttpFSServer implementation of the FileSystemAccess FileSystem.
  * <p/>
  * This implementation allows a user to access HDFS over HTTP via a HttpFSServer server.
  */
-public class HttpFSFileSystem extends FileSystem {
+public class HttpFSFileSystem extends FileSystem
+  implements DelegationTokenRenewer.Renewable {
 
-  public static final String SERVICE_NAME = "/webhdfs";
+  public static final String SERVICE_NAME = HttpFSUtils.SERVICE_NAME;
 
-  public static final String SERVICE_VERSION = "/v1";
+  public static final String SERVICE_VERSION = HttpFSUtils.SERVICE_VERSION;
 
-  public static final String SERVICE_PREFIX = SERVICE_NAME + SERVICE_VERSION;
+  public static final String SCHEME = "webhdfs";
 
   public static final String OP_PARAM = "op";
   public static final String DO_AS_PARAM = "doas";
@@ -84,7 +89,6 @@ public class HttpFSFileSystem extends FileSystem {
   public static final String GROUP_PARAM = "group";
   public static final String MODIFICATION_TIME_PARAM = "modificationtime";
   public static final String ACCESS_TIME_PARAM = "accesstime";
-  public static final String RENEWER_PARAM = "renewer";
 
   public static final Short DEFAULT_PERMISSION = 0755;
 
@@ -144,9 +148,6 @@ public class HttpFSFileSystem extends FileSystem {
   public static final String CONTENT_SUMMARY_SPACE_CONSUMED_JSON = "spaceConsumed";
   public static final String CONTENT_SUMMARY_SPACE_QUOTA_JSON = "spaceQuota";
 
-  public static final String DELEGATION_TOKEN_JSON = "Token";
-  public static final String DELEGATION_TOKEN_URL_STRING_JSON = "urlString";
-
   public static final String ERROR_JSON = "RemoteException";
   public static final String ERROR_EXCEPTION_JSON = "exception";
   public static final String ERROR_CLASSNAME_JSON = "javaClassName";
@@ -184,8 +185,31 @@ public class HttpFSFileSystem extends FileSystem {
 
   private AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
   private URI uri;
+  private InetSocketAddress httpFSAddr;
   private Path workingDir;
+  private UserGroupInformation realUser;
   private String doAs;
+  private Token<?> delegationToken;
+
+  //This method enables handling UGI doAs with SPNEGO, we have to
+  //fallback to the realuser who logged in with Kerberos credentials
+  private <T> T doAsRealUserIfNecessary(final Callable<T> callable)
+    throws IOException {
+    try {
+      if (realUser.getShortUserName().equals(doAs)) {
+        return callable.call();
+      } else {
+        return realUser.doAs(new PrivilegedExceptionAction<T>() {
+          @Override
+          public T run() throws Exception {
+            return callable.call();
+          }
+        });
+      }
+    } catch (Exception ex) {
+      throw new IOException(ex.toString(), ex);
+    }
+  }
 
   /**
    * Convenience method that creates a <code>HttpURLConnection</code> for the
@@ -204,25 +228,23 @@ public class HttpFSFileSystem extends FileSystem {
    *
    * @throws IOException thrown if an IO error occurrs.
    */
-  private HttpURLConnection getConnection(String method, Map<String, String> params,
-                                          Path path, boolean makeQualified) throws IOException {
-    params.put(DO_AS_PARAM, doAs);
+  private HttpURLConnection getConnection(final String method,
+      Map<String, String> params, Path path, boolean makeQualified)
+      throws IOException {
+    if (!realUser.getShortUserName().equals(doAs)) {
+      params.put(DO_AS_PARAM, doAs);
+    }
+    HttpFSKerberosAuthenticator.injectDelegationToken(params, delegationToken);
     if (makeQualified) {
       path = makeQualified(path);
     }
-    URI uri = path.toUri();
-    StringBuilder sb = new StringBuilder();
-    sb.append(uri.getScheme()).append("://").append(uri.getAuthority()).
-      append(SERVICE_PREFIX).append(uri.getPath());
-
-    String separator = "?";
-    for (Map.Entry<String, String> entry : params.entrySet()) {
-      sb.append(separator).append(entry.getKey()).append("=").
-        append(URLEncoder.encode(entry.getValue(), "UTF8"));
-      separator = "&";
-    }
-    URL url = new URL(sb.toString());
-    return getConnection(url, method);
+    final URL url = HttpFSUtils.createHttpURL(path, params);
+    return doAsRealUserIfNecessary(new Callable<HttpURLConnection>() {
+      @Override
+      public HttpURLConnection call() throws Exception {
+        return getConnection(url, method);
+      }
+    });
   }
 
   /**
@@ -240,7 +262,8 @@ public class HttpFSFileSystem extends FileSystem {
    */
   private HttpURLConnection getConnection(URL url, String method) throws IOException {
     Class<? extends Authenticator> klass =
-      getConf().getClass("httpfs.authenticator.class", HttpKerberosAuthenticator.class, Authenticator.class);
+      getConf().getClass("httpfs.authenticator.class",
+                         HttpFSKerberosAuthenticator.class, Authenticator.class);
     Authenticator authenticator = ReflectionUtils.newInstance(klass, getConf());
     try {
       HttpURLConnection conn = new AuthenticatedURL(authenticator).openConnection(url, authToken);
@@ -254,63 +277,6 @@ public class HttpFSFileSystem extends FileSystem {
     }
   }
 
-  /**
-   * Convenience method that JSON Parses the <code>InputStream</code> of a <code>HttpURLConnection</code>.
-   *
-   * @param conn the <code>HttpURLConnection</code>.
-   *
-   * @return the parsed JSON object.
-   *
-   * @throws IOException thrown if the <code>InputStream</code> could not be JSON parsed.
-   */
-  private static Object jsonParse(HttpURLConnection conn) throws IOException {
-    try {
-      JSONParser parser = new JSONParser();
-      return parser.parse(new InputStreamReader(conn.getInputStream()));
-    } catch (ParseException ex) {
-      throw new IOException("JSON parser error, " + ex.getMessage(), ex);
-    }
-  }
-
-  /**
-   * Validates the status of an <code>HttpURLConnection</code> against an expected HTTP
-   * status code. If the current status code is not the expected one it throws an exception
-   * with a detail message using Server side error messages if available.
-   *
-   * @param conn the <code>HttpURLConnection</code>.
-   * @param expected the expected HTTP status code.
-   *
-   * @throws IOException thrown if the current status code does not match the expected one.
-   */
-  private static void validateResponse(HttpURLConnection conn, int expected) throws IOException {
-    int status = conn.getResponseCode();
-    if (status != expected) {
-      try {
-        JSONObject json = (JSONObject) jsonParse(conn);
-        json = (JSONObject) json.get(ERROR_JSON);
-        String message = (String) json.get(ERROR_MESSAGE_JSON);
-        String exception = (String) json.get(ERROR_EXCEPTION_JSON);
-        String className = (String) json.get(ERROR_CLASSNAME_JSON);
-
-        try {
-          ClassLoader cl = HttpFSFileSystem.class.getClassLoader();
-          Class klass = cl.loadClass(className);
-          Constructor constr = klass.getConstructor(String.class);
-          throw (IOException) constr.newInstance(message);
-        } catch (IOException ex) {
-          throw ex;
-        } catch (Exception ex) {
-          throw new IOException(MessageFormat.format("{0} - {1}", exception, message));
-        }
-      } catch (IOException ex) {
-        if (ex.getCause() instanceof IOException) {
-          throw (IOException) ex.getCause();
-        }
-        throw new IOException(MessageFormat.format("HTTP status [{0}], {1}", status, conn.getResponseMessage()));
-      }
-    }
-  }
-
   /**
    * Called after a new FileSystem instance is constructed.
    *
@@ -320,15 +286,28 @@ public class HttpFSFileSystem extends FileSystem {
   @Override
   public void initialize(URI name, Configuration conf) throws IOException {
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    doAs = ugi.getUserName();
+
+    //the real use is the one that has the Kerberos credentials needed for
+    //SPNEGO to work
+    realUser = ugi.getRealUser();
+    if (realUser == null) {
+      realUser = UserGroupInformation.getLoginUser();
+    }
+    doAs = ugi.getShortUserName();
     super.initialize(name, conf);
     try {
-      uri = new URI(name.getScheme() + "://" + name.getHost() + ":" + name.getPort());
+      uri = new URI(name.getScheme() + "://" + name.getAuthority());
+      httpFSAddr = NetUtils.createSocketAddr(getCanonicalUri().toString());
     } catch (URISyntaxException ex) {
       throw new IOException(ex);
     }
   }
 
+  @Override
+  public String getScheme() {
+    return SCHEME;
+  }
+
   /**
    * Returns a URI whose scheme and authority identify this FileSystem.
    *
@@ -339,6 +318,16 @@ public class HttpFSFileSystem extends FileSystem {
     return uri;
   }
 
+  /**
+   * Get the default port for this file system.
+   * @return the default port or 0 if there isn't one
+   */
+  @Override
+  protected int getDefaultPort() {
+    return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
+        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT);
+  }
+
   /**
    * HttpFSServer subclass of the <code>FSDataInputStream</code>.
    * <p/>
@@ -397,7 +386,7 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(OP_PARAM, Operation.OPEN.toString());
     HttpURLConnection conn = getConnection(Operation.OPEN.getMethod(), params,
                                            f, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
     return new FSDataInputStream(
       new HttpFSDataInputStream(conn.getInputStream(), bufferSize));
   }
@@ -424,7 +413,7 @@ public class HttpFSFileSystem extends FileSystem {
       try {
         super.close();
       } finally {
-        validateResponse(conn, closeStatus);
+        HttpFSUtils.validateResponse(conn, closeStatus);
       }
     }
 
@@ -460,11 +449,11 @@ public class HttpFSFileSystem extends FileSystem {
             OutputStream os = new BufferedOutputStream(conn.getOutputStream(), bufferSize);
             return new HttpFSDataOutputStream(conn, os, expectedStatus, statistics);
           } catch (IOException ex) {
-            validateResponse(conn, expectedStatus);
+            HttpFSUtils.validateResponse(conn, expectedStatus);
             throw ex;
           }
         } else {
-          validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
+          HttpFSUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
           throw new IOException("Missing HTTP 'Location' header for [" + conn.getURL() + "]");
         }
       } else {
@@ -476,7 +465,7 @@ public class HttpFSFileSystem extends FileSystem {
       if (exceptionAlreadyHandled) {
         throw ex;
       } else {
-        validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
+        HttpFSUtils.validateResponse(conn, HTTP_TEMPORARY_REDIRECT);
         throw ex;
       }
     }
@@ -548,8 +537,8 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(DESTINATION_PARAM, dst.toString());
     HttpURLConnection conn = getConnection(Operation.RENAME.getMethod(),
                                            params, src, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
-    JSONObject json = (JSONObject) jsonParse(conn);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(RENAME_JSON);
   }
 
@@ -584,8 +573,8 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(RECURSIVE_PARAM, Boolean.toString(recursive));
     HttpURLConnection conn = getConnection(Operation.DELETE.getMethod(),
                                            params, f, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
-    JSONObject json = (JSONObject) jsonParse(conn);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(DELETE_JSON);
   }
 
@@ -605,8 +594,8 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(OP_PARAM, Operation.LISTSTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.LISTSTATUS.getMethod(),
                                            params, f, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
-    JSONObject json = (JSONObject) jsonParse(conn);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(FILE_STATUSES_JSON);
     JSONArray jsonArray = (JSONArray) json.get(FILE_STATUS_JSON);
     FileStatus[] array = new FileStatus[jsonArray.size()];
@@ -653,8 +642,8 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(PERMISSION_PARAM, permissionToString(permission));
     HttpURLConnection conn = getConnection(Operation.MKDIRS.getMethod(),
                                            params, f, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
-    JSONObject json = (JSONObject) jsonParse(conn);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(MKDIRS_JSON);
   }
 
@@ -674,8 +663,8 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(OP_PARAM, Operation.GETFILESTATUS.toString());
     HttpURLConnection conn = getConnection(Operation.GETFILESTATUS.getMethod(),
                                            params, f, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
-    JSONObject json = (JSONObject) jsonParse(conn);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     json = (JSONObject) json.get(FILE_STATUS_JSON);
     f = makeQualified(f);
     return createFileStatus(f, json);
@@ -693,8 +682,8 @@ public class HttpFSFileSystem extends FileSystem {
       HttpURLConnection conn =
         getConnection(Operation.GETHOMEDIRECTORY.getMethod(), params,
                       new Path(getUri().toString(), "/"), false);
-      validateResponse(conn, HttpURLConnection.HTTP_OK);
-      JSONObject json = (JSONObject) jsonParse(conn);
+      HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+      JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
       return new Path((String) json.get(HOME_DIR_JSON));
     } catch (IOException ex) {
       throw new RuntimeException(ex);
@@ -718,7 +707,7 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(GROUP_PARAM, groupname);
     HttpURLConnection conn = getConnection(Operation.SETOWNER.getMethod(),
                                            params, p, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -733,7 +722,7 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(OP_PARAM, Operation.SETPERMISSION.toString());
     params.put(PERMISSION_PARAM, permissionToString(permission));
     HttpURLConnection conn = getConnection(Operation.SETPERMISSION.getMethod(), params, p, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -755,7 +744,7 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(ACCESS_TIME_PARAM, Long.toString(atime));
     HttpURLConnection conn = getConnection(Operation.SETTIMES.getMethod(),
                                            params, p, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
   }
 
   /**
@@ -777,19 +766,11 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(REPLICATION_PARAM, Short.toString(replication));
     HttpURLConnection conn =
       getConnection(Operation.SETREPLICATION.getMethod(), params, src, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
-    JSONObject json = (JSONObject) jsonParse(conn);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
     return (Boolean) json.get(SET_REPLICATION_JSON);
   }
 
-  /**
-   * Creates a <code>FileStatus</code> object using a JSON file-status payload
-   * received from a HttpFSServer server.
-   *
-   * @param json a JSON file-status payload received from a HttpFSServer server
-   *
-   * @return the corresponding <code>FileStatus</code>
-   */
   private FileStatus createFileStatus(Path parent, JSONObject json) {
     String pathSuffix = (String) json.get(PATH_SUFFIX_JSON);
     Path path = (pathSuffix.equals("")) ? parent : new Path(parent, pathSuffix);
@@ -828,9 +809,9 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(OP_PARAM, Operation.GETCONTENTSUMMARY.toString());
     HttpURLConnection conn =
       getConnection(Operation.GETCONTENTSUMMARY.getMethod(), params, f, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
-    JSONObject json =
-      (JSONObject) ((JSONObject) jsonParse(conn)).get(CONTENT_SUMMARY_JSON);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) ((JSONObject)
+      HttpFSUtils.jsonParse(conn)).get(CONTENT_SUMMARY_JSON);
     return new ContentSummary((Long) json.get(CONTENT_SUMMARY_LENGTH_JSON),
                               (Long) json.get(CONTENT_SUMMARY_FILE_COUNT_JSON),
                               (Long) json.get(CONTENT_SUMMARY_DIRECTORY_COUNT_JSON),
@@ -846,9 +827,9 @@ public class HttpFSFileSystem extends FileSystem {
     params.put(OP_PARAM, Operation.GETFILECHECKSUM.toString());
     HttpURLConnection conn =
       getConnection(Operation.GETFILECHECKSUM.getMethod(), params, f, true);
-    validateResponse(conn, HttpURLConnection.HTTP_OK);
-    final JSONObject json =
-      (JSONObject) ((JSONObject) jsonParse(conn)).get(FILE_CHECKSUM_JSON);
+    HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    final JSONObject json = (JSONObject) ((JSONObject)
+      HttpFSUtils.jsonParse(conn)).get(FILE_CHECKSUM_JSON);
     return new FileChecksum() {
       @Override
       public String getAlgorithmName() {
@@ -877,4 +858,56 @@ public class HttpFSFileSystem extends FileSystem {
     };
   }
 
+
+  @Override
+  @SuppressWarnings("deprecation")
+  public Token<?> getDelegationToken(final String renewer)
+    throws IOException {
+    return doAsRealUserIfNecessary(new Callable<Token<?>>() {
+      @Override
+      public Token<?> call() throws Exception {
+        return HttpFSKerberosAuthenticator.
+          getDelegationToken(uri, httpFSAddr, authToken, renewer);
+      }
+    });
+  }
+
+
+  @Override
+  public List<Token<?>> getDelegationTokens(final String renewer)
+    throws IOException {
+    return doAsRealUserIfNecessary(new Callable<List<Token<?>>>() {
+      @Override
+      public List<Token<?>> call() throws Exception {
+        return HttpFSKerberosAuthenticator.
+          getDelegationTokens(uri, httpFSAddr, authToken, renewer);
+      }
+    });
+  }
+
+  public long renewDelegationToken(final Token<?> token) throws IOException {
+    return doAsRealUserIfNecessary(new Callable<Long>() {
+      @Override
+      public Long call() throws Exception {
+        return HttpFSKerberosAuthenticator.
+          renewDelegationToken(uri,  authToken, token);
+      }
+    });
+  }
+
+  public void cancelDelegationToken(final Token<?> token) throws IOException {
+    HttpFSKerberosAuthenticator.
+      cancelDelegationToken(uri, authToken, token);
+  }
+
+  @Override
+  public Token<?> getRenewToken() {
+    return delegationToken;
+  }
+
+  @Override
+  public <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
+    delegationToken = token;
+  }
+
 }

+ 226 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java

@@ -0,0 +1,226 @@
+/**
+ * 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.fs.http.client;
+
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.Authenticator;
+import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A <code>KerberosAuthenticator</code> subclass that fallback to
+ * {@link HttpFSPseudoAuthenticator}.
+ */
+public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
+
+  /**
+   * Returns the fallback authenticator if the server does not use
+   * Kerberos SPNEGO HTTP authentication.
+   *
+   * @return a {@link HttpFSPseudoAuthenticator} instance.
+   */
+  @Override
+  protected Authenticator getFallBackAuthenticator() {
+    return new HttpFSPseudoAuthenticator();
+  }
+
+  private static final String HTTP_GET = "GET";
+  private static final String HTTP_PUT = "PUT";
+
+  public static final String DELEGATION_PARAM = "delegation";
+  public static final String TOKEN_PARAM = "token";
+  public static final String RENEWER_PARAM = "renewer";
+  public static final String TOKEN_KIND = "HTTPFS_DELEGATION_TOKEN";
+  public static final String DELEGATION_TOKEN_JSON = "Token";
+  public static final String DELEGATION_TOKENS_JSON = "Tokens";
+  public static final String DELEGATION_TOKEN_URL_STRING_JSON = "urlString";
+  public static final String RENEW_DELEGATION_TOKEN_JSON = "long";
+
+  /**
+   * DelegationToken operations.
+   */
+  public static enum DelegationTokenOperation {
+    GETDELEGATIONTOKEN(HTTP_GET, true),
+    GETDELEGATIONTOKENS(HTTP_GET, true),
+    RENEWDELEGATIONTOKEN(HTTP_PUT, true),
+    CANCELDELEGATIONTOKEN(HTTP_PUT, false);
+
+    private String httpMethod;
+    private boolean requiresKerberosCredentials;
+
+    private DelegationTokenOperation(String httpMethod,
+                                     boolean requiresKerberosCredentials) {
+      this.httpMethod = httpMethod;
+      this.requiresKerberosCredentials = requiresKerberosCredentials;
+    }
+
+    public String getHttpMethod() {
+      return httpMethod;
+    }
+
+    public boolean requiresKerberosCredentials() {
+      return requiresKerberosCredentials;
+    }
+
+  }
+
+  public static void injectDelegationToken(Map<String, String> params,
+                                          Token<?> dtToken)
+    throws IOException {
+    if (dtToken != null) {
+      params.put(DELEGATION_PARAM, dtToken.encodeToUrlString());
+    }
+  }
+
+  private boolean hasDelegationToken(URL url) {
+    return url.getQuery().contains(DELEGATION_PARAM + "=");
+  }
+
+  @Override
+  public void authenticate(URL url, AuthenticatedURL.Token token)
+    throws IOException, AuthenticationException {
+    if (!hasDelegationToken(url)) {
+      super.authenticate(url, token);
+    }
+  }
+
+  public static final String OP_PARAM = "op";
+
+  private static List<Token<?>> getDelegationTokens(URI fsURI,
+    InetSocketAddress httpFSAddr, DelegationTokenOperation op,
+    AuthenticatedURL.Token token, String renewer)
+    throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM, op.toString());
+    params.put(RENEWER_PARAM,renewer);
+    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    AuthenticatedURL aUrl =
+      new AuthenticatedURL(new HttpFSKerberosAuthenticator());
+    try {
+      HttpURLConnection conn = aUrl.openConnection(url, token);
+      conn.setRequestMethod(op.getHttpMethod());
+      HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+      List<String> list = new ArrayList<String>();
+      if (op == DelegationTokenOperation.GETDELEGATIONTOKEN) {
+        JSONObject json = (JSONObject) ((JSONObject)
+          HttpFSUtils.jsonParse(conn)).get(DELEGATION_TOKEN_JSON);
+        String tokenStr = (String)
+          json.get(DELEGATION_TOKEN_URL_STRING_JSON);
+        list.add(tokenStr);
+      }
+      else if (op == DelegationTokenOperation.GETDELEGATIONTOKENS) {
+        JSONObject json = (JSONObject) ((JSONObject)
+          HttpFSUtils.jsonParse(conn)).get(DELEGATION_TOKENS_JSON);
+        JSONArray array = (JSONArray) json.get(DELEGATION_TOKEN_JSON);
+        for (Object element : array) {
+          String tokenStr = (String)
+            ((Map) element).get(DELEGATION_TOKEN_URL_STRING_JSON);
+          list.add(tokenStr);
+        }
+
+      } else {
+        throw new IllegalArgumentException("Invalid operation: " +
+                                           op.toString());
+      }
+      List<Token<?>> dTokens = new ArrayList<Token<?>>();
+      for (String tokenStr : list) {
+        Token<AbstractDelegationTokenIdentifier> dToken =
+          new Token<AbstractDelegationTokenIdentifier>();
+        dToken.decodeFromUrlString(tokenStr);
+        dTokens.add(dToken);
+        SecurityUtil.setTokenService(dToken, httpFSAddr);
+      }
+      return dTokens;
+    } catch (AuthenticationException ex) {
+      throw new IOException(ex.toString(), ex);
+    }
+  }
+
+  public static List<Token<?>> getDelegationTokens(URI fsURI,
+    InetSocketAddress httpFSAddr, AuthenticatedURL.Token token,
+    String renewer) throws IOException {
+    return getDelegationTokens(fsURI, httpFSAddr,
+      DelegationTokenOperation.GETDELEGATIONTOKENS, token, renewer);
+  }
+
+  public static Token<?> getDelegationToken(URI fsURI,
+    InetSocketAddress httpFSAddr, AuthenticatedURL.Token token,
+    String renewer) throws IOException {
+    return getDelegationTokens(fsURI, httpFSAddr,
+      DelegationTokenOperation.GETDELEGATIONTOKENS, token, renewer).get(0);
+  }
+
+  public static long renewDelegationToken(URI fsURI,
+    AuthenticatedURL.Token token, Token<?> dToken) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM,
+               DelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
+    params.put(TOKEN_PARAM, dToken.encodeToUrlString());
+    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    AuthenticatedURL aUrl =
+      new AuthenticatedURL(new HttpFSKerberosAuthenticator());
+    try {
+      HttpURLConnection conn = aUrl.openConnection(url, token);
+      conn.setRequestMethod(
+        DelegationTokenOperation.RENEWDELEGATIONTOKEN.getHttpMethod());
+      HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+      JSONObject json = (JSONObject) ((JSONObject)
+        HttpFSUtils.jsonParse(conn)).get(DELEGATION_TOKEN_JSON);
+      return (Long)(json.get(RENEW_DELEGATION_TOKEN_JSON));
+    } catch (AuthenticationException ex) {
+      throw new IOException(ex.toString(), ex);
+    }
+  }
+
+  public static void cancelDelegationToken(URI fsURI,
+    AuthenticatedURL.Token token, Token<?> dToken) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(OP_PARAM,
+               DelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
+    params.put(TOKEN_PARAM, dToken.encodeToUrlString());
+    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    AuthenticatedURL aUrl =
+      new AuthenticatedURL(new HttpFSKerberosAuthenticator());
+    try {
+      HttpURLConnection conn = aUrl.openConnection(url, token);
+      conn.setRequestMethod(
+        DelegationTokenOperation.CANCELDELEGATIONTOKEN.getHttpMethod());
+      HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    } catch (AuthenticationException ex) {
+      throw new IOException(ex.toString(), ex);
+    }
+  }
+
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpPseudoAuthenticator.java → hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSPseudoAuthenticator.java

@@ -27,7 +27,7 @@ import java.io.IOException;
  * A <code>PseudoAuthenticator</code> subclass that uses FileSystemAccess's
  * <code>UserGroupInformation</code> to obtain the client user name (the UGI's login user).
  */
-public class HttpPseudoAuthenticator extends PseudoAuthenticator {
+public class HttpFSPseudoAuthenticator extends PseudoAuthenticator {
 
   /**
    * Return the client user name.

+ 148 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java

@@ -0,0 +1,148 @@
+/**
+ * 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.fs.http.client;
+
+import org.apache.hadoop.fs.Path;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.lang.reflect.Constructor;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.text.MessageFormat;
+import java.util.Map;
+
+/**
+ * Utility methods used by HttpFS classes.
+ */
+public class HttpFSUtils {
+
+  public static final String SERVICE_NAME = "/webhdfs";
+
+  public static final String SERVICE_VERSION = "/v1";
+
+  private static final String SERVICE_PATH = SERVICE_NAME + SERVICE_VERSION;
+
+  /**
+   * Convenience method that creates an HTTP <code>URL</code> for the
+   * HttpFSServer file system operations.
+   * <p/>
+   *
+   * @param path the file path.
+   * @param params the query string parameters.
+   *
+   * @return a <code>URL</code> for the HttpFSServer server,
+   *
+   * @throws IOException thrown if an IO error occurrs.
+   */
+  static URL createHttpURL(Path path, Map<String, String> params)
+    throws IOException {
+    URI uri = path.toUri();
+    String realScheme;
+    if (uri.getScheme().equalsIgnoreCase(HttpFSFileSystem.SCHEME)) {
+      realScheme = "http";
+    } else {
+      throw new IllegalArgumentException(MessageFormat.format(
+        "Invalid scheme [{0}] it should be 'webhdfs'", uri));
+    }
+    StringBuilder sb = new StringBuilder();
+    sb.append(realScheme).append("://").append(uri.getAuthority()).
+      append(SERVICE_PATH).append(uri.getPath());
+
+    String separator = "?";
+    for (Map.Entry<String, String> entry : params.entrySet()) {
+      sb.append(separator).append(entry.getKey()).append("=").
+        append(URLEncoder.encode(entry.getValue(), "UTF8"));
+      separator = "&";
+    }
+    return new URL(sb.toString());
+  }
+
+  /**
+   * Validates the status of an <code>HttpURLConnection</code> against an
+   * expected HTTP status code. If the current status code is not the expected
+   * one it throws an exception with a detail message using Server side error
+   * messages if available.
+   *
+   * @param conn the <code>HttpURLConnection</code>.
+   * @param expected the expected HTTP status code.
+   *
+   * @throws IOException thrown if the current status code does not match the
+   * expected one.
+   */
+  @SuppressWarnings({"unchecked", "deprecation"})
+  static void validateResponse(HttpURLConnection conn, int expected)
+    throws IOException {
+    int status = conn.getResponseCode();
+    if (status != expected) {
+      try {
+        JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+        json = (JSONObject) json.get(HttpFSFileSystem.ERROR_JSON);
+        String message = (String) json.get(HttpFSFileSystem.ERROR_MESSAGE_JSON);
+        String exception = (String)
+          json.get(HttpFSFileSystem.ERROR_EXCEPTION_JSON);
+        String className = (String)
+          json.get(HttpFSFileSystem.ERROR_CLASSNAME_JSON);
+
+        try {
+          ClassLoader cl = HttpFSFileSystem.class.getClassLoader();
+          Class klass = cl.loadClass(className);
+          Constructor constr = klass.getConstructor(String.class);
+          throw (IOException) constr.newInstance(message);
+        } catch (IOException ex) {
+          throw ex;
+        } catch (Exception ex) {
+          throw new IOException(MessageFormat.format("{0} - {1}", exception,
+                                                     message));
+        }
+      } catch (IOException ex) {
+        if (ex.getCause() instanceof IOException) {
+          throw (IOException) ex.getCause();
+        }
+        throw new IOException(
+          MessageFormat.format("HTTP status [{0}], {1}",
+                               status, conn.getResponseMessage()));
+      }
+    }
+  }
+
+  /**
+   * Convenience method that JSON Parses the <code>InputStream</code> of a
+   * <code>HttpURLConnection</code>.
+   *
+   * @param conn the <code>HttpURLConnection</code>.
+   *
+   * @return the parsed JSON object.
+   *
+   * @throws IOException thrown if the <code>InputStream</code> could not be
+   * JSON parsed.
+   */
+  static Object jsonParse(HttpURLConnection conn) throws IOException {
+    try {
+      JSONParser parser = new JSONParser();
+      return parser.parse(new InputStreamReader(conn.getInputStream()));
+    } catch (ParseException ex) {
+      throw new IOException("JSON parser error, " + ex.getMessage(), ex);
+    }
+  }
+}

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/AuthFilter.java → hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.fs.http.server;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
-
 import javax.servlet.FilterConfig;
 import java.io.FileReader;
 import java.io.IOException;
@@ -31,7 +30,7 @@ import java.util.Properties;
  * Subclass of hadoop-auth <code>AuthenticationFilter</code> that obtains its configuration
  * from HttpFSServer's server configuration.
  */
-public class AuthFilter extends AuthenticationFilter {
+public class HttpFSAuthenticationFilter extends AuthenticationFilter {
   private static final String CONF_PREFIX = "httpfs.authentication.";
 
   private static final String SIGNATURE_SECRET_FILE = SIGNATURE_SECRET + ".file";
@@ -63,6 +62,11 @@ public class AuthFilter extends AuthenticationFilter {
       }
     }
 
+    if (props.getProperty(AUTH_TYPE).equals("kerberos")) {
+      props.setProperty(AUTH_TYPE,
+                        HttpFSKerberosAuthenticationHandler.class.getName());
+    }
+
     String signatureSecretFile = props.getProperty(SIGNATURE_SECRET_FILE, null);
     if (signatureSecretFile == null) {
       throw new RuntimeException("Undefined property: " + SIGNATURE_SECRET_FILE);
@@ -84,5 +88,4 @@ public class AuthFilter extends AuthenticationFilter {
     return props;
   }
 
-
 }

+ 255 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandler.java

@@ -0,0 +1,255 @@
+/**
+ * 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.fs.http.server;
+
+import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator.DelegationTokenOperation;
+import org.apache.hadoop.lib.service.DelegationTokenIdentifier;
+import org.apache.hadoop.lib.service.DelegationTokenManager;
+import org.apache.hadoop.lib.service.DelegationTokenManagerException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
+import org.apache.hadoop.security.token.Token;
+import org.json.simple.JSONObject;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.io.Writer;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Server side <code>AuthenticationHandler</code> that authenticates requests
+ * using the incoming delegation token as a 'delegation' query string parameter.
+ * <p/>
+ * If not delegation token is present in the request it delegates to the
+ * {@link KerberosAuthenticationHandler}
+ */
+public class HttpFSKerberosAuthenticationHandler
+  extends KerberosAuthenticationHandler {
+
+  static final Set<String> DELEGATION_TOKEN_OPS =
+    new HashSet<String>();
+
+  static {
+    DELEGATION_TOKEN_OPS.add(
+      DelegationTokenOperation.GETDELEGATIONTOKEN.toString());
+    DELEGATION_TOKEN_OPS.add(
+      DelegationTokenOperation.GETDELEGATIONTOKENS.toString());
+    DELEGATION_TOKEN_OPS.add(
+      DelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
+    DELEGATION_TOKEN_OPS.add(
+      DelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
+  }
+
+  public static final String TYPE = "kerberos-dt";
+
+  /**
+   * Returns authentication type of the handler.
+   *
+   * @return <code>delegationtoken-kerberos</code>
+   */
+  @Override
+  public String getType() {
+    return TYPE;
+  }
+
+  private static final String ENTER = System.getProperty("line.separator");
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public boolean managementOperation(AuthenticationToken token,
+      HttpServletRequest request, HttpServletResponse response)
+    throws IOException, AuthenticationException {
+    boolean requestContinues = true;
+    String op = request.getParameter(HttpFSFileSystem.OP_PARAM);
+    op = (op != null) ? op.toUpperCase() : null;
+    if (DELEGATION_TOKEN_OPS.contains(op) &&
+        !request.getMethod().equals("OPTIONS")) {
+      DelegationTokenOperation dtOp =
+        DelegationTokenOperation.valueOf(op);
+      if (dtOp.getHttpMethod().equals(request.getMethod())) {
+        if (dtOp.requiresKerberosCredentials() && token == null) {
+          response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
+            MessageFormat.format(
+              "Operation [{0}] requires SPNEGO authentication established",
+              dtOp));
+          requestContinues = false;
+        } else {
+          DelegationTokenManager tokenManager =
+            HttpFSServerWebApp.get().get(DelegationTokenManager.class);
+          try {
+            Map map = null;
+            switch (dtOp) {
+              case GETDELEGATIONTOKEN:
+              case GETDELEGATIONTOKENS:
+                String renewerParam =
+                  request.getParameter(HttpFSKerberosAuthenticator.RENEWER_PARAM);
+                if (renewerParam == null) {
+                  renewerParam = token.getUserName();
+                }
+                Token<?> dToken = tokenManager.createToken(
+                  UserGroupInformation.getCurrentUser(), renewerParam);
+                if (dtOp == DelegationTokenOperation.GETDELEGATIONTOKEN) {
+                  map = delegationTokenToJSON(dToken);
+                } else {
+                  map = delegationTokensToJSON(Arrays.asList((Token)dToken));
+                }
+                break;
+              case RENEWDELEGATIONTOKEN:
+              case CANCELDELEGATIONTOKEN:
+                String tokenParam =
+                  request.getParameter(HttpFSKerberosAuthenticator.TOKEN_PARAM);
+                if (tokenParam == null) {
+                  response.sendError(HttpServletResponse.SC_BAD_REQUEST,
+                    MessageFormat.format(
+                      "Operation [{0}] requires the parameter [{1}]",
+                      dtOp, HttpFSKerberosAuthenticator.TOKEN_PARAM));
+                  requestContinues = false;
+                } else {
+                  if (dtOp == DelegationTokenOperation.CANCELDELEGATIONTOKEN) {
+                    Token<DelegationTokenIdentifier> dt =
+                      new Token<DelegationTokenIdentifier>();
+                    dt.decodeFromUrlString(tokenParam);
+                    tokenManager.cancelToken(dt,
+                      UserGroupInformation.getCurrentUser().getUserName());
+                  } else {
+                    Token<DelegationTokenIdentifier> dt =
+                      new Token<DelegationTokenIdentifier>();
+                    dt.decodeFromUrlString(tokenParam);
+                    long expirationTime =
+                      tokenManager.renewToken(dt, token.getUserName());
+                    map = new HashMap();
+                    map.put("long", expirationTime);
+                  }
+                }
+                break;
+            }
+            if (requestContinues) {
+              response.setStatus(HttpServletResponse.SC_OK);
+              if (map != null) {
+                response.setContentType(MediaType.APPLICATION_JSON);
+                Writer writer = response.getWriter();
+                JSONObject.writeJSONString(map, writer);
+                writer.write(ENTER);
+                writer.flush();
+
+              }
+              requestContinues = false;
+            }
+          } catch (DelegationTokenManagerException ex) {
+            throw new AuthenticationException(ex.toString(), ex);
+          }
+        }
+      } else {
+        response.sendError(HttpServletResponse.SC_BAD_REQUEST,
+          MessageFormat.format(
+            "Wrong HTTP method [{0}] for operation [{1}], it should be [{2}]",
+            request.getMethod(), dtOp, dtOp.getHttpMethod()));
+        requestContinues = false;
+      }
+    }
+    return requestContinues;
+  }
+
+  @SuppressWarnings("unchecked")
+  private static Map delegationTokenToJSON(Token token) throws IOException {
+    Map json = new LinkedHashMap();
+    json.put(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON,
+             token.encodeToUrlString());
+    Map response = new LinkedHashMap();
+    response.put(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON, json);
+    return response;
+  }
+  
+  @SuppressWarnings("unchecked")
+  private static Map delegationTokensToJSON(List<Token> tokens)
+    throws IOException {
+    List list = new ArrayList();
+    for (Token token : tokens) {
+      Map map = new HashMap();
+      map.put(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON,
+              token.encodeToUrlString());
+      list.add(map);
+    }
+    Map map = new HashMap();
+    map.put(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON, list);
+    Map response = new LinkedHashMap();
+    response.put(HttpFSKerberosAuthenticator.DELEGATION_TOKENS_JSON, map);
+    return response;
+  }
+
+  /**
+   * Authenticates a request looking for the <code>delegation</code>
+   * query-string parameter and verifying it is a valid token. If there is not
+   * <code>delegation</code> query-string parameter, it delegates the
+   * authentication to the {@link KerberosAuthenticationHandler} unless it is
+   * disabled.
+   *
+   * @param request the HTTP client request.
+   * @param response the HTTP client response.
+   *
+   * @return the authentication token for the authenticated request.
+   * @throws IOException thrown if an IO error occurred.
+   * @throws AuthenticationException thrown if the authentication failed.
+   */
+  @Override
+  public AuthenticationToken authenticate(HttpServletRequest request,
+                                          HttpServletResponse response)
+    throws IOException, AuthenticationException {
+    AuthenticationToken token;
+    String delegationParam =
+      request.getParameter(HttpFSKerberosAuthenticator.DELEGATION_PARAM);
+    if (delegationParam != null) {
+      try {
+        Token<DelegationTokenIdentifier> dt =
+          new Token<DelegationTokenIdentifier>();
+        dt.decodeFromUrlString(delegationParam);
+        DelegationTokenManager tokenManager =
+          HttpFSServerWebApp.get().get(DelegationTokenManager.class);
+        UserGroupInformation ugi = tokenManager.verifyToken(dt);
+        final String shortName = ugi.getShortUserName();
+
+        // creating a ephemeral token
+        token = new AuthenticationToken(shortName, ugi.getUserName(),
+                                        getType());
+        token.setExpires(0);
+      } catch (Throwable ex) {
+        throw new AuthenticationException("Could not verify DelegationToken, " +
+                                          ex.toString(), ex);
+      }
+    } else {
+      token = super.authenticate(request, response);
+    }
+    return token;
+  }
+
+
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java

@@ -70,7 +70,7 @@ public class HttpFSServerWebApp extends ServerWebApp {
   /**
    * Constructor used for testing purposes.
    */
-  protected HttpFSServerWebApp(String homeDir, String configDir, String logDir,
+  public HttpFSServerWebApp(String homeDir, String configDir, String logDir,
                                String tempDir, Configuration config) {
     super(NAME, homeDir, configDir, logDir, tempDir, config);
   }

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/server/ServerException.java

@@ -39,7 +39,11 @@ public class ServerException extends XException {
     S08("Could not load service classes, {0}"),
     S09("Could not set service [{0}] programmatically -server shutting down-, {1}"),
     S10("Service [{0}] requires service [{1}]"),
-    S11("Service [{0}] exception during status change to [{1}] -server shutting down-, {2}");
+    S11("Service [{0}] exception during status change to [{1}] -server shutting down-, {2}"),
+    S12("Could not start service [{0}], {1}"),
+    S13("Missing system property [{0}]"),
+    S14("Could not initialize server, {0}")
+    ;
 
     private String msg;
 

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenIdentifier.java

@@ -0,0 +1,57 @@
+/**
+ * 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.lib.service;
+
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+
+/**
+ * HttpFS <code>DelegationTokenIdentifier</code> implementation.
+ */
+public class DelegationTokenIdentifier
+  extends AbstractDelegationTokenIdentifier {
+
+  public static final Text KIND_NAME =
+    new Text(HttpFSKerberosAuthenticator.TOKEN_KIND);
+
+  public DelegationTokenIdentifier() {
+  }
+
+  /**
+   * Create a new delegation token identifier
+   *
+   * @param owner the effective username of the token owner
+   * @param renewer the username of the renewer
+   * @param realUser the real username of the token owner
+   */
+  public DelegationTokenIdentifier(Text owner, Text renewer, Text realUser) {
+    super(owner, renewer, realUser);
+  }
+
+
+  /**
+   * Returns the kind, <code>TOKEN_KIND</code>.
+   * @return returns <code>TOKEN_KIND</code>.
+   */
+  @Override
+  public Text getKind() {
+    return KIND_NAME;
+  }
+
+}

+ 76 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenManager.java

@@ -0,0 +1,76 @@
+/**
+ * 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.lib.service;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * Service interface to manage HttpFS delegation tokens.
+ */
+public interface DelegationTokenManager {
+
+  /**
+   * Creates a delegation token.
+   *
+   * @param ugi UGI creating the token.
+   * @param renewer token renewer.
+   * @return new delegation token.
+   * @throws DelegationTokenManagerException thrown if the token could not be
+   * created.
+   */
+  public Token<DelegationTokenIdentifier> createToken(UserGroupInformation ugi,
+                                                      String renewer)
+    throws DelegationTokenManagerException;
+
+  /**
+   * Renews a delegation token.
+   *
+   * @param token delegation token to renew.
+   * @param renewer token renewer.
+   * @return epoc expiration time.
+   * @throws DelegationTokenManagerException thrown if the token could not be
+   * renewed.
+   */
+  public long renewToken(Token<DelegationTokenIdentifier> token, String renewer)
+    throws DelegationTokenManagerException;
+
+  /**
+   * Cancels a delegation token.
+   *
+   * @param token delegation token to cancel.
+   * @param canceler token canceler.
+   * @throws DelegationTokenManagerException thrown if the token could not be
+   * canceled.
+   */
+  public void cancelToken(Token<DelegationTokenIdentifier> token,
+                          String canceler)
+    throws DelegationTokenManagerException;
+
+  /**
+   * Verifies a delegation token.
+   *
+   * @param token delegation token to verify.
+   * @return the UGI for the token.
+   * @throws DelegationTokenManagerException thrown if the token could not be
+   * verified.
+   */
+  public UserGroupInformation verifyToken(Token<DelegationTokenIdentifier> token)
+    throws DelegationTokenManagerException;
+
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenManagerException.java

@@ -0,0 +1,49 @@
+/**
+ * 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.lib.service;
+
+import org.apache.hadoop.lib.lang.XException;
+
+/**
+ * Exception thrown by the {@link DelegationTokenManager} service implementation.
+ */
+public class DelegationTokenManagerException extends XException {
+
+  public enum ERROR implements XException.ERROR {
+    DT01("Could not verify delegation token, {0}"),
+    DT02("Could not renew delegation token, {0}"),
+    DT03("Could not cancel delegation token, {0}"),
+    DT04("Could not create delegation token, {0}");
+
+    private String template;
+
+    ERROR(String template) {
+      this.template = template;
+    }
+
+    @Override
+    public String getTemplate() {
+      return template;
+    }
+  }
+
+  public DelegationTokenManagerException(ERROR error, Object... params) {
+    super(error, params);
+  }
+
+}

+ 231 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/security/DelegationTokenManagerService.java

@@ -0,0 +1,231 @@
+/**
+ * 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.lib.service.security;
+
+import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.lib.server.BaseService;
+import org.apache.hadoop.lib.server.ServerException;
+import org.apache.hadoop.lib.server.ServiceException;
+import org.apache.hadoop.lib.service.DelegationTokenIdentifier;
+import org.apache.hadoop.lib.service.DelegationTokenManager;
+import org.apache.hadoop.lib.service.DelegationTokenManagerException;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * DelegationTokenManager service implementation.
+ */
+public class DelegationTokenManagerService extends BaseService
+  implements DelegationTokenManager {
+
+  private static final String PREFIX = "delegation.token.manager";
+
+  private static final String UPDATE_INTERVAL = "update.interval";
+
+  private static final String MAX_LIFETIME = "max.lifetime";
+
+  private static final String RENEW_INTERVAL = "renew.interval";
+
+  private static final long HOUR = 60 * 60 * 1000;
+  private static final long DAY = 24 * HOUR;
+
+  DelegationTokenSecretManager secretManager = null;
+
+  public DelegationTokenManagerService() {
+    super(PREFIX);
+  }
+
+  /**
+   * Initializes the service.
+   *
+   * @throws ServiceException thrown if the service could not be initialized.
+   */
+  @Override
+  protected void init() throws ServiceException {
+
+    long updateInterval = getServiceConfig().getLong(UPDATE_INTERVAL, DAY);
+    long maxLifetime = getServiceConfig().getLong(MAX_LIFETIME, 7 * DAY);
+    long renewInterval = getServiceConfig().getLong(RENEW_INTERVAL, DAY);
+    secretManager = new DelegationTokenSecretManager(updateInterval,
+                                                     maxLifetime,
+                                                     renewInterval, HOUR);
+    try {
+      secretManager.startThreads();
+    } catch (IOException ex) {
+      throw new ServiceException(ServiceException.ERROR.S12,
+                                 DelegationTokenManager.class.getSimpleName(),
+                                 ex.toString(), ex);
+    }
+  }
+
+  /**
+   * Destroys the service.
+   */
+  @Override
+  public void destroy() {
+    secretManager.stopThreads();
+    super.destroy();
+  }
+
+  /**
+   * Returns the service interface.
+   *
+   * @return the service interface.
+   */
+  @Override
+  public Class getInterface() {
+    return DelegationTokenManager.class;
+  }
+
+  /**
+   * Creates a delegation token.
+   *
+   * @param ugi UGI creating the token.
+   * @param renewer token renewer.
+   * @return new delegation token.
+   * @throws DelegationTokenManagerException thrown if the token could not be
+   * created.
+   */
+  @Override
+  public Token<DelegationTokenIdentifier> createToken(UserGroupInformation ugi,
+                                                      String renewer)
+    throws DelegationTokenManagerException {
+    renewer = (renewer == null) ? ugi.getShortUserName() : renewer;
+    String user = ugi.getUserName();
+    Text owner = new Text(user);
+    Text realUser = null;
+    if (ugi.getRealUser() != null) {
+      realUser = new Text(ugi.getRealUser().getUserName());
+    }
+    DelegationTokenIdentifier tokenIdentifier =
+      new DelegationTokenIdentifier(owner, new Text(renewer), realUser);
+    Token<DelegationTokenIdentifier> token =
+      new Token<DelegationTokenIdentifier>(tokenIdentifier, secretManager);
+    try {
+      SecurityUtil.setTokenService(token,
+                                   HttpFSServerWebApp.get().getAuthority());
+    } catch (ServerException ex) {
+      throw new DelegationTokenManagerException(
+        DelegationTokenManagerException.ERROR.DT04, ex.toString(), ex);
+    }
+    return token;
+  }
+
+  /**
+   * Renews a delegation token.
+   *
+   * @param token delegation token to renew.
+   * @param renewer token renewer.
+   * @return epoc expiration time.
+   * @throws DelegationTokenManagerException thrown if the token could not be
+   * renewed.
+   */
+  @Override
+  public long renewToken(Token<DelegationTokenIdentifier> token, String renewer)
+    throws DelegationTokenManagerException {
+    try {
+      return secretManager.renewToken(token, renewer);
+    } catch (IOException ex) {
+      throw new DelegationTokenManagerException(
+        DelegationTokenManagerException.ERROR.DT02, ex.toString(), ex);
+    }
+  }
+
+  /**
+   * Cancels a delegation token.
+   *
+   * @param token delegation token to cancel.
+   * @param canceler token canceler.
+   * @throws DelegationTokenManagerException thrown if the token could not be
+   * canceled.
+   */
+  @Override
+  public void cancelToken(Token<DelegationTokenIdentifier> token,
+                          String canceler)
+    throws DelegationTokenManagerException {
+    try {
+      secretManager.cancelToken(token, canceler);
+    } catch (IOException ex) {
+      throw new DelegationTokenManagerException(
+        DelegationTokenManagerException.ERROR.DT03, ex.toString(), ex);
+    }
+  }
+
+  /**
+   * Verifies a delegation token.
+   *
+   * @param token delegation token to verify.
+   * @return the UGI for the token.
+   * @throws DelegationTokenManagerException thrown if the token could not be
+   * verified.
+   */
+  @Override
+  public UserGroupInformation verifyToken(Token<DelegationTokenIdentifier> token)
+    throws DelegationTokenManagerException {
+    ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
+    DataInputStream dis = new DataInputStream(buf);
+    DelegationTokenIdentifier id = new DelegationTokenIdentifier();
+    try {
+      id.readFields(dis);
+      dis.close();
+      secretManager.verifyToken(id, token.getPassword());
+    } catch (Exception ex) {
+      throw new DelegationTokenManagerException(
+        DelegationTokenManagerException.ERROR.DT01, ex.toString(), ex);
+    }
+    return id.getUser();
+  }
+
+  private static class DelegationTokenSecretManager
+    extends AbstractDelegationTokenSecretManager<DelegationTokenIdentifier> {
+
+    /**
+     * Create a secret manager
+     *
+     * @param delegationKeyUpdateInterval the number of seconds for rolling new
+     * secret keys.
+     * @param delegationTokenMaxLifetime the maximum lifetime of the delegation
+     * tokens
+     * @param delegationTokenRenewInterval how often the tokens must be renewed
+     * @param delegationTokenRemoverScanInterval how often the tokens are
+     * scanned
+     * for expired tokens
+     */
+    public DelegationTokenSecretManager(long delegationKeyUpdateInterval,
+                                        long delegationTokenMaxLifetime,
+                                        long delegationTokenRenewInterval,
+                                        long delegationTokenRemoverScanInterval) {
+      super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
+            delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
+    }
+
+    @Override
+    public DelegationTokenIdentifier createIdentifier() {
+      return new DelegationTokenIdentifier();
+    }
+
+  }
+
+}

+ 65 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java

@@ -18,12 +18,16 @@
 
 package org.apache.hadoop.lib.servlet;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.lib.server.Server;
 import org.apache.hadoop.lib.server.ServerException;
 
 import javax.servlet.ServletContextEvent;
 import javax.servlet.ServletContextListener;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
 import java.text.MessageFormat;
 
 /**
@@ -36,9 +40,13 @@ public abstract class ServerWebApp extends Server implements ServletContextListe
   private static final String CONFIG_DIR = ".config.dir";
   private static final String LOG_DIR = ".log.dir";
   private static final String TEMP_DIR = ".temp.dir";
+  private static final String HTTP_HOSTNAME = ".http.hostname";
+  private static final String HTTP_PORT = ".http.port";
 
   private static ThreadLocal<String> HOME_DIR_TL = new ThreadLocal<String>();
 
+  private InetSocketAddress authority;
+
   /**
    * Method for testing purposes.
    */
@@ -146,6 +154,38 @@ public abstract class ServerWebApp extends Server implements ServletContextListe
     }
   }
 
+  /**
+   * Resolves the host & port InetSocketAddress the web server is listening to.
+   * <p/>
+   * This implementation looks for the following 2 properties:
+   * <ul>
+   *   <li>#SERVER_NAME#.http.hostname</li>
+   *   <li>#SERVER_NAME#.http.port</li>
+   * </ul>
+   *
+   * @return the host & port InetSocketAddress the web server is listening to.
+   * @throws ServerException thrown if any of the above 2 properties is not defined.
+   */
+  protected InetSocketAddress resolveAuthority() throws ServerException {
+    String hostnameKey = getName() + HTTP_HOSTNAME;
+    String portKey = getName() + HTTP_PORT;
+    String host = System.getProperty(hostnameKey);
+    String port = System.getProperty(portKey);
+    if (host == null) {
+      throw new ServerException(ServerException.ERROR.S13, hostnameKey);
+    }
+    if (port == null) {
+      throw new ServerException(ServerException.ERROR.S13, portKey);
+    }
+    try {
+      InetAddress add = InetAddress.getByName(hostnameKey);
+      int portNum = Integer.parseInt(port);
+      return new InetSocketAddress(add, portNum);
+    } catch (UnknownHostException ex) {
+      throw new ServerException(ServerException.ERROR.S14, ex.toString(), ex);
+    }
+  }
+
   /**
    * Destroys the <code>ServletContextListener</code> which destroys
    * the Server.
@@ -156,4 +196,29 @@ public abstract class ServerWebApp extends Server implements ServletContextListe
     destroy();
   }
 
+  /**
+   * Returns the hostname:port InetSocketAddress the webserver is listening to.
+   *
+   * @return the hostname:port InetSocketAddress the webserver is listening to.
+   */
+  public InetSocketAddress getAuthority() throws ServerException {
+    synchronized (this) {
+      if (authority == null) {
+          authority = resolveAuthority();
+      }
+    }
+    return authority;
+  }
+
+  /**
+   * Sets an alternate hostname:port InetSocketAddress to use.
+   * <p/>
+   * For testing purposes.
+   * 
+   * @param authority alterante authority.
+   */
+  @VisibleForTesting
+  public void setAuthority(InetSocketAddress authority) {
+    this.authority = authority;
+  }
 }

+ 29 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml

@@ -35,6 +35,7 @@
       org.apache.hadoop.lib.service.scheduler.SchedulerService,
       org.apache.hadoop.lib.service.security.GroupsService,
       org.apache.hadoop.lib.service.security.ProxyUserService,
+      org.apache.hadoop.lib.service.security.DelegationTokenManagerService,
       org.apache.hadoop.lib.service.hadoop.FileSystemAccessService
     </value>
     <description>
@@ -88,12 +89,12 @@
     <description>
       Defines the authentication mechanism used by httpfs for its HTTP clients.
 
-      Valid values are 'simple' and 'kerberos'.
+      Valid values are 'simple' or 'kerberos'.
 
       If using 'simple' HTTP clients must specify the username with the
       'user.name' query string parameter.
 
-      If using 'kerberos' HTTP clients must use HTTP SPNEGO.
+      If using 'kerberos' HTTP clients must use HTTP SPNEGO or delegation tokens.
     </description>
   </property>
 
@@ -153,6 +154,32 @@
     </description>
   </property>
 
+  <!-- HttpFS Delegation Token configuration -->
+
+  <property>
+    <name>httpfs.delegation.token.manager.update.interval</name>
+    <value>86400</value>
+    <description>
+      HttpFS delegation token update interval, default 1 day, in seconds.
+    </description>
+  </property>
+
+  <property>
+    <name>httpfs.delegation.token.manager.max.lifetime</name>
+    <value>604800</value>
+    <description>
+      HttpFS delegation token maximum lifetime, default 7 days, in seconds
+    </description>
+  </property>
+
+  <property>
+    <name>httpfs.delegation.token.manager.renewal.interval</name>
+    <value>86400</value>
+    <description>
+      HttpFS delegation token update interval, default 1 day, in seconds.
+    </description>
+  </property>
+
   <!-- FileSystemAccess Namenode Security Configuration -->
 
   <property>

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/webapp/WEB-INF/web.xml

@@ -47,7 +47,7 @@
 
   <filter>
     <filter-name>authFilter</filter-name>
-    <filter-class>org.apache.hadoop.fs.http.server.AuthFilter</filter-class>
+    <filter-class>org.apache.hadoop.fs.http.server.HttpFSAuthenticationFilter</filter-class>
   </filter>
 
   <filter>

+ 14 - 4
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java

@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Writer;
+import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
@@ -100,16 +101,24 @@ public class TestHttpFSFileSystem extends HFSTestCase {
     server.start();
   }
 
+  protected Class getFileSystemClass() {
+    return HttpFSFileSystem.class;
+  }
+
   protected FileSystem getHttpFileSystem() throws Exception {
     Configuration conf = new Configuration();
-    conf.set("fs.http.impl", HttpFSFileSystem.class.getName());
-    return FileSystem.get(TestJettyHelper.getJettyURL().toURI(), conf);
+    conf.set("fs.webhdfs.impl", getFileSystemClass().getName());
+    URI uri = new URI("webhdfs://" +
+                      TestJettyHelper.getJettyURL().toURI().getAuthority());
+    return FileSystem.get(uri, conf);
   }
 
   protected void testGet() throws Exception {
     FileSystem fs = getHttpFileSystem();
     Assert.assertNotNull(fs);
-    Assert.assertEquals(fs.getUri(), TestJettyHelper.getJettyURL().toURI());
+    URI uri = new URI("webhdfs://" +
+                      TestJettyHelper.getJettyURL().toURI().getAuthority());
+    Assert.assertEquals(fs.getUri(), uri);
     fs.close();
   }
 
@@ -474,8 +483,9 @@ public class TestHttpFSFileSystem extends HFSTestCase {
     for (int i = 0; i < Operation.values().length; i++) {
       ops[i] = new Object[]{Operation.values()[i]};
     }
+    //To test one or a subset of operations do:
+    //return Arrays.asList(new Object[][]{ new Object[]{Operation.OPEN}});
     return Arrays.asList(ops);
-//    return Arrays.asList(new Object[][]{ new Object[]{Operation.CREATE}});
   }
 
   private Operation operation;

+ 513 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystem.java.orig

@@ -0,0 +1,513 @@
+/**
+ * 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.fs.http.client;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Writer;
+import java.net.URL;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.HFSTestCase;
+import org.apache.hadoop.test.HadoopUsersConfTestHelper;
+import org.apache.hadoop.test.TestDir;
+import org.apache.hadoop.test.TestDirHelper;
+import org.apache.hadoop.test.TestHdfs;
+import org.apache.hadoop.test.TestHdfsHelper;
+import org.apache.hadoop.test.TestJetty;
+import org.apache.hadoop.test.TestJettyHelper;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.webapp.WebAppContext;
+
+@RunWith(value = Parameterized.class)
+public class TestHttpFSFileSystem extends HFSTestCase {
+
+  private void createHttpFSServer() throws Exception {
+    File homeDir = TestDirHelper.getTestDir();
+    Assert.assertTrue(new File(homeDir, "conf").mkdir());
+    Assert.assertTrue(new File(homeDir, "log").mkdir());
+    Assert.assertTrue(new File(homeDir, "temp").mkdir());
+    HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath());
+
+    File secretFile = new File(new File(homeDir, "conf"), "secret");
+    Writer w = new FileWriter(secretFile);
+    w.write("secret");
+    w.close();
+
+    //HDFS configuration
+    String fsDefaultName = TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
+    Configuration conf = new Configuration(false);
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
+    File hdfsSite = new File(new File(homeDir, "conf"), "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    //HTTPFS configuration
+    conf = new Configuration(false);
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".groups",
+             HadoopUsersConfTestHelper.getHadoopProxyUserGroups());
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts",
+             HadoopUsersConfTestHelper.getHadoopProxyUserHosts());
+    conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
+    File httpfsSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
+    os = new FileOutputStream(httpfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    URL url = cl.getResource("webapp");
+    WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
+    Server server = TestJettyHelper.getJettyServer();
+    server.addHandler(context);
+    server.start();
+  }
+
+  protected FileSystem getHttpFileSystem() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("fs.http.impl", HttpFSFileSystem.class.getName());
+    return FileSystem.get(TestJettyHelper.getJettyURL().toURI(), conf);
+  }
+
+  protected void testGet() throws Exception {
+    FileSystem fs = getHttpFileSystem();
+    Assert.assertNotNull(fs);
+    Assert.assertEquals(fs.getUri(), TestJettyHelper.getJettyURL().toURI());
+    fs.close();
+  }
+
+  private void testOpen() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    OutputStream os = fs.create(path);
+    os.write(1);
+    os.close();
+    fs.close();
+    fs = getHttpFileSystem();
+    InputStream is = fs.open(new Path(path.toUri().getPath()));
+    Assert.assertEquals(is.read(), 1);
+    is.close();
+    fs.close();
+  }
+
+  private void testCreate(Path path, boolean override) throws Exception {
+    FileSystem fs = getHttpFileSystem();
+    FsPermission permission = new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE);
+    OutputStream os = fs.create(new Path(path.toUri().getPath()), permission, override, 1024,
+                                (short) 2, 100 * 1024 * 1024, null);
+    os.write(1);
+    os.close();
+    fs.close();
+
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    FileStatus status = fs.getFileStatus(path);
+    Assert.assertEquals(status.getReplication(), 2);
+    Assert.assertEquals(status.getBlockSize(), 100 * 1024 * 1024);
+    Assert.assertEquals(status.getPermission(), permission);
+    InputStream is = fs.open(path);
+    Assert.assertEquals(is.read(), 1);
+    is.close();
+    fs.close();
+  }
+
+  private void testCreate() throws Exception {
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    testCreate(path, false);
+    testCreate(path, true);
+    try {
+      testCreate(path, false);
+      Assert.fail();
+    } catch (IOException ex) {
+
+    } catch (Exception ex) {
+      Assert.fail();
+    }
+  }
+
+  private void testAppend() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    OutputStream os = fs.create(path);
+    os.write(1);
+    os.close();
+    fs.close();
+    fs = getHttpFileSystem();
+    os = fs.append(new Path(path.toUri().getPath()));
+    os.write(2);
+    os.close();
+    fs.close();
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    InputStream is = fs.open(path);
+    Assert.assertEquals(is.read(), 1);
+    Assert.assertEquals(is.read(), 2);
+    Assert.assertEquals(is.read(), -1);
+    is.close();
+    fs.close();
+  }
+
+  private void testRename() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo");
+    fs.mkdirs(path);
+    fs.close();
+    fs = getHttpFileSystem();
+    Path oldPath = new Path(path.toUri().getPath());
+    Path newPath = new Path(path.getParent(), "bar");
+    fs.rename(oldPath, newPath);
+    fs.close();
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Assert.assertFalse(fs.exists(oldPath));
+    Assert.assertTrue(fs.exists(newPath));
+    fs.close();
+  }
+
+  private void testDelete() throws Exception {
+    Path foo = new Path(TestHdfsHelper.getHdfsTestDir(), "foo");
+    Path bar = new Path(TestHdfsHelper.getHdfsTestDir(), "bar");
+    Path foe = new Path(TestHdfsHelper.getHdfsTestDir(), "foe");
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    fs.mkdirs(foo);
+    fs.mkdirs(new Path(bar, "a"));
+    fs.mkdirs(foe);
+
+    FileSystem hoopFs = getHttpFileSystem();
+    Assert.assertTrue(hoopFs.delete(new Path(foo.toUri().getPath()), false));
+    Assert.assertFalse(fs.exists(foo));
+    try {
+      hoopFs.delete(new Path(bar.toUri().getPath()), false);
+      Assert.fail();
+    } catch (IOException ex) {
+    } catch (Exception ex) {
+      Assert.fail();
+    }
+    Assert.assertTrue(fs.exists(bar));
+    Assert.assertTrue(hoopFs.delete(new Path(bar.toUri().getPath()), true));
+    Assert.assertFalse(fs.exists(bar));
+
+    Assert.assertTrue(fs.exists(foe));
+    Assert.assertTrue(hoopFs.delete(foe, true));
+    Assert.assertFalse(fs.exists(foe));
+
+    hoopFs.close();
+    fs.close();
+  }
+
+  private void testListStatus() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    OutputStream os = fs.create(path);
+    os.write(1);
+    os.close();
+    FileStatus status1 = fs.getFileStatus(path);
+    fs.close();
+
+    fs = getHttpFileSystem();
+    FileStatus status2 = fs.getFileStatus(new Path(path.toUri().getPath()));
+    fs.close();
+
+    Assert.assertEquals(status2.getPermission(), status1.getPermission());
+    Assert.assertEquals(status2.getPath().toUri().getPath(), status1.getPath().toUri().getPath());
+    Assert.assertEquals(status2.getReplication(), status1.getReplication());
+    Assert.assertEquals(status2.getBlockSize(), status1.getBlockSize());
+    Assert.assertEquals(status2.getAccessTime(), status1.getAccessTime());
+    Assert.assertEquals(status2.getModificationTime(), status1.getModificationTime());
+    Assert.assertEquals(status2.getOwner(), status1.getOwner());
+    Assert.assertEquals(status2.getGroup(), status1.getGroup());
+    Assert.assertEquals(status2.getLen(), status1.getLen());
+
+    FileStatus[] stati = fs.listStatus(path.getParent());
+    Assert.assertEquals(stati.length, 1);
+    Assert.assertEquals(stati[0].getPath().getName(), path.getName());
+  }
+
+  private void testWorkingdirectory() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path workingDir = fs.getWorkingDirectory();
+    fs.close();
+
+    fs = getHttpFileSystem();
+    Path hoopWorkingDir = fs.getWorkingDirectory();
+    fs.close();
+    Assert.assertEquals(hoopWorkingDir.toUri().getPath(), workingDir.toUri().getPath());
+
+    fs = getHttpFileSystem();
+    fs.setWorkingDirectory(new Path("/tmp"));
+    workingDir = fs.getWorkingDirectory();
+    fs.close();
+    Assert.assertEquals(workingDir.toUri().getPath(), new Path("/tmp").toUri().getPath());
+  }
+
+  private void testMkdirs() throws Exception {
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo");
+    FileSystem fs = getHttpFileSystem();
+    fs.mkdirs(path);
+    fs.close();
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Assert.assertTrue(fs.exists(path));
+    fs.close();
+  }
+
+  private void testSetTimes() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    OutputStream os = fs.create(path);
+    os.write(1);
+    os.close();
+    FileStatus status1 = fs.getFileStatus(path);
+    fs.close();
+    long at = status1.getAccessTime();
+    long mt = status1.getModificationTime();
+
+    fs = getHttpFileSystem();
+    fs.setTimes(path, mt + 10, at + 20);
+    fs.close();
+
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    status1 = fs.getFileStatus(path);
+    fs.close();
+    long atNew = status1.getAccessTime();
+    long mtNew = status1.getModificationTime();
+    Assert.assertEquals(mtNew, mt + 10);
+    Assert.assertEquals(atNew, at + 20);
+  }
+
+  private void testSetPermission() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foodir");
+    fs.mkdirs(path);
+
+    fs = getHttpFileSystem();
+    FsPermission permission1 = new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE);
+    fs.setPermission(path, permission1);
+    fs.close();
+
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    FileStatus status1 = fs.getFileStatus(path);
+    fs.close();
+    FsPermission permission2 = status1.getPermission();
+    Assert.assertEquals(permission2, permission1);
+
+    //sticky bit 
+    fs = getHttpFileSystem();
+    permission1 = new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE, true);
+    fs.setPermission(path, permission1);
+    fs.close();
+
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    status1 = fs.getFileStatus(path);
+    fs.close();
+    permission2 = status1.getPermission();
+    Assert.assertTrue(permission2.getStickyBit());
+    Assert.assertEquals(permission2, permission1);
+  }
+
+  private void testSetOwner() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    OutputStream os = fs.create(path);
+    os.write(1);
+    os.close();
+    fs.close();
+
+    fs = getHttpFileSystem();
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[1];
+    String group = HadoopUsersConfTestHelper.getHadoopUserGroups(user)[0];
+    fs.setOwner(path, user, group);
+    fs.close();
+
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    FileStatus status1 = fs.getFileStatus(path);
+    fs.close();
+    Assert.assertEquals(status1.getOwner(), user);
+    Assert.assertEquals(status1.getGroup(), group);
+  }
+
+  private void testSetReplication() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    OutputStream os = fs.create(path);
+    os.write(1);
+    os.close();
+    fs.close();
+    fs.setReplication(path, (short) 2);
+
+    fs = getHttpFileSystem();
+    fs.setReplication(path, (short) 1);
+    fs.close();
+
+    fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    FileStatus status1 = fs.getFileStatus(path);
+    fs.close();
+    Assert.assertEquals(status1.getReplication(), (short) 1);
+  }
+
+  private void testChecksum() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    OutputStream os = fs.create(path);
+    os.write(1);
+    os.close();
+    FileChecksum hdfsChecksum = fs.getFileChecksum(path);
+    fs.close();
+    fs = getHttpFileSystem();
+    FileChecksum httpChecksum = fs.getFileChecksum(path);
+    fs.close();
+    Assert.assertEquals(httpChecksum.getAlgorithmName(), hdfsChecksum.getAlgorithmName());
+    Assert.assertEquals(httpChecksum.getLength(), hdfsChecksum.getLength());
+    Assert.assertArrayEquals(httpChecksum.getBytes(), hdfsChecksum.getBytes());
+  }
+
+  private void testContentSummary() throws Exception {
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path path = new Path(TestHdfsHelper.getHdfsTestDir(), "foo.txt");
+    OutputStream os = fs.create(path);
+    os.write(1);
+    os.close();
+    ContentSummary hdfsContentSummary = fs.getContentSummary(path);
+    fs.close();
+    fs = getHttpFileSystem();
+    ContentSummary httpContentSummary = fs.getContentSummary(path);
+    fs.close();
+    Assert.assertEquals(httpContentSummary.getDirectoryCount(), hdfsContentSummary.getDirectoryCount());
+    Assert.assertEquals(httpContentSummary.getFileCount(), hdfsContentSummary.getFileCount());
+    Assert.assertEquals(httpContentSummary.getLength(), hdfsContentSummary.getLength());
+    Assert.assertEquals(httpContentSummary.getQuota(), hdfsContentSummary.getQuota());
+    Assert.assertEquals(httpContentSummary.getSpaceConsumed(), hdfsContentSummary.getSpaceConsumed());
+    Assert.assertEquals(httpContentSummary.getSpaceQuota(), hdfsContentSummary.getSpaceQuota());
+  }
+
+  protected enum Operation {
+    GET, OPEN, CREATE, APPEND, RENAME, DELETE, LIST_STATUS, WORKING_DIRECTORY, MKDIRS,
+    SET_TIMES, SET_PERMISSION, SET_OWNER, SET_REPLICATION, CHECKSUM, CONTENT_SUMMARY
+  }
+
+  private void operation(Operation op) throws Exception {
+    switch (op) {
+      case GET:
+        testGet();
+        break;
+      case OPEN:
+        testOpen();
+        break;
+      case CREATE:
+        testCreate();
+        break;
+      case APPEND:
+        testAppend();
+        break;
+      case RENAME:
+        testRename();
+        break;
+      case DELETE:
+        testDelete();
+        break;
+      case LIST_STATUS:
+        testListStatus();
+        break;
+      case WORKING_DIRECTORY:
+        testWorkingdirectory();
+        break;
+      case MKDIRS:
+        testMkdirs();
+        break;
+      case SET_TIMES:
+        testSetTimes();
+        break;
+      case SET_PERMISSION:
+        testSetPermission();
+        break;
+      case SET_OWNER:
+        testSetOwner();
+        break;
+      case SET_REPLICATION:
+        testSetReplication();
+        break;
+      case CHECKSUM:
+        testChecksum();
+        break;
+      case CONTENT_SUMMARY:
+        testContentSummary();
+        break;
+    }
+  }
+
+  @Parameterized.Parameters
+  public static Collection operations() {
+    Object[][] ops = new Object[Operation.values().length][];
+    for (int i = 0; i < Operation.values().length; i++) {
+      ops[i] = new Object[]{Operation.values()[i]};
+    }
+    return Arrays.asList(ops);
+//    return Arrays.asList(new Object[][]{ new Object[]{Operation.CREATE}});
+  }
+
+  private Operation operation;
+
+  public TestHttpFSFileSystem(Operation operation) {
+    this.operation = operation;
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testOperation() throws Exception {
+    createHttpFSServer();
+    operation(operation);
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testOperationDoAs() throws Exception {
+    createHttpFSServer();
+    UserGroupInformation ugi = UserGroupInformation.createProxyUser(HadoopUsersConfTestHelper.getHadoopUsers()[0],
+                                                                    UserGroupInformation.getCurrentUser());
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        operation(operation);
+        return null;
+      }
+    });
+  }
+
+}

+ 2 - 14
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestWebhdfsFileSystem.java

@@ -36,20 +36,8 @@ public class TestWebhdfsFileSystem extends TestHttpFSFileSystem {
   }
 
   @Override
-  protected FileSystem getHttpFileSystem() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set("fs.webhdfs.impl", WebHdfsFileSystem.class.getName());
-    URI uri = new URI("webhdfs://" + TestJettyHelper.getJettyURL().toURI().getAuthority());
-    return FileSystem.get(uri, conf);
-  }
-
-  @Override
-  protected void testGet() throws Exception {
-    FileSystem fs = getHttpFileSystem();
-    Assert.assertNotNull(fs);
-    URI uri = new URI("webhdfs://" + TestJettyHelper.getJettyURL().toURI().getAuthority());
-    Assert.assertEquals(fs.getUri(), uri);
-    fs.close();
+  protected Class getFileSystemClass() {
+    return WebHdfsFileSystem.class;
   }
 
 }

+ 11 - 17
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpKerberosAuthenticator.java → hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandlerForTesting.java

@@ -15,27 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.hadoop.fs.http.server;
 
-package org.apache.hadoop.fs.http.client;
+import javax.servlet.ServletException;
+import java.util.Properties;
 
+public class HttpFSKerberosAuthenticationHandlerForTesting
+  extends HttpFSKerberosAuthenticationHandler {
 
-import org.apache.hadoop.security.authentication.client.Authenticator;
-import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
-
-/**
- * A <code>KerberosAuthenticator</code> subclass that fallback to
- * {@link HttpPseudoAuthenticator}.
- */
-public class HttpKerberosAuthenticator extends KerberosAuthenticator {
+  @Override
+  public void init(Properties config) throws ServletException {
+    //NOP overwrite to avoid Kerberos initialization
+  }
 
-  /**
-   * Returns the fallback authenticator if the server does not use
-   * Kerberos SPNEGO HTTP authentication.
-   *
-   * @return a {@link HttpPseudoAuthenticator} instance.
-   */
   @Override
-  protected Authenticator getFallBackAuthenticator() {
-    return new HttpPseudoAuthenticator();
+  public void destroy() {
+    //NOP overwrite to avoid Kerberos initialization
   }
 }

+ 310 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java

@@ -0,0 +1,310 @@
+/**
+ * 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.fs.http.server;
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator.DelegationTokenOperation;
+import org.apache.hadoop.lib.service.DelegationTokenIdentifier;
+import org.apache.hadoop.lib.service.DelegationTokenManager;
+import org.apache.hadoop.lib.service.DelegationTokenManagerException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.HFSTestCase;
+import org.apache.hadoop.test.TestDir;
+import org.apache.hadoop.test.TestDirHelper;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.MediaType;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+
+public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
+
+  @Test
+  @TestDir
+  public void testManagementOperations() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+
+    Configuration httpfsConf = new Configuration(false);
+    HttpFSServerWebApp server =
+      new HttpFSServerWebApp(dir, dir, dir, dir, httpfsConf);
+    server.setAuthority(new InetSocketAddress(InetAddress.getLocalHost(), 
+                                              14000));
+    AuthenticationHandler handler =
+      new HttpFSKerberosAuthenticationHandlerForTesting();
+    try {
+      server.init();
+      handler.init(null);
+
+      testNonManagementOperation(handler);
+      testManagementOperationErrors(handler);
+      testGetToken(handler, false, null);
+      testGetToken(handler, true, null);
+      testGetToken(handler, false, "foo");
+      testGetToken(handler, true, "foo");
+      testCancelToken(handler);
+      testRenewToken(handler);
+
+    } finally {
+      if (handler != null) {
+        handler.destroy();
+      }
+    server.destroy();
+    }
+  }
+
+  private void testNonManagementOperation(AuthenticationHandler handler)
+    throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(request.getParameter(HttpFSFileSystem.OP_PARAM)).
+      thenReturn(null);
+    Assert.assertTrue(handler.managementOperation(null, request, null));
+    Mockito.when(request.getParameter(HttpFSFileSystem.OP_PARAM)).
+      thenReturn(HttpFSFileSystem.Operation.CREATE.toString());
+    Assert.assertTrue(handler.managementOperation(null, request, null));
+  }
+
+  private void testManagementOperationErrors(AuthenticationHandler handler)
+    throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(request.getParameter(HttpFSFileSystem.OP_PARAM)).
+      thenReturn(DelegationTokenOperation.GETDELEGATIONTOKEN.toString());
+    Mockito.when(request.getMethod()).thenReturn("FOO");
+    Assert.assertFalse(handler.managementOperation(null, request, response));
+    Mockito.verify(response).sendError(
+      Mockito.eq(HttpServletResponse.SC_BAD_REQUEST),
+      Mockito.startsWith("Wrong HTTP method"));
+
+    Mockito.reset(response);
+    Mockito.when(request.getMethod()).
+      thenReturn(DelegationTokenOperation.GETDELEGATIONTOKEN.getHttpMethod());
+    Assert.assertFalse(handler.managementOperation(null, request, response));
+    Mockito.verify(response).sendError(
+      Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED),
+      Mockito.contains("requires SPNEGO"));
+  }
+
+  private void testGetToken(AuthenticationHandler handler, boolean tokens,
+                            String renewer)
+    throws Exception {
+    DelegationTokenOperation op =
+      (tokens) ? DelegationTokenOperation.GETDELEGATIONTOKENS
+               : DelegationTokenOperation.GETDELEGATIONTOKEN;
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(request.getParameter(HttpFSFileSystem.OP_PARAM)).
+      thenReturn(op.toString());
+    Mockito.when(request.getMethod()).
+      thenReturn(op.getHttpMethod());
+
+    AuthenticationToken token = Mockito.mock(AuthenticationToken.class);
+    Mockito.when(token.getUserName()).thenReturn("user");
+    Assert.assertFalse(handler.managementOperation(null, request, response));
+    Mockito.when(request.getParameter(HttpFSKerberosAuthenticator.RENEWER_PARAM)).
+      thenReturn(renewer);
+
+    Mockito.reset(response);
+    StringWriter writer = new StringWriter();
+    PrintWriter pwriter = new PrintWriter(writer);
+    Mockito.when(response.getWriter()).thenReturn(pwriter);
+    Assert.assertFalse(handler.managementOperation(token, request, response));
+    if (renewer == null) {
+      Mockito.verify(token).getUserName();
+    } else {
+      Mockito.verify(token, Mockito.never()).getUserName();
+    }
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
+    Mockito.verify(response).setContentType(MediaType.APPLICATION_JSON);
+    pwriter.close();
+    String responseOutput = writer.toString();
+    String tokenLabel = (tokens)
+                        ? HttpFSKerberosAuthenticator.DELEGATION_TOKENS_JSON
+                        : HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON;
+    if (tokens) {
+      Assert.assertTrue(responseOutput.contains(tokenLabel));
+    } else {
+      Assert.assertTrue(responseOutput.contains(tokenLabel));
+    }
+    Assert.assertTrue(responseOutput.contains(
+      HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON));
+    JSONObject json = (JSONObject) new JSONParser().parse(responseOutput);
+    json = (JSONObject) json.get(tokenLabel);
+    String tokenStr;
+    if (tokens) {
+      json = (JSONObject) ((JSONArray)
+        json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON)).get(0);
+    }
+    tokenStr = (String)
+      json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
+    Token<DelegationTokenIdentifier> dt = new Token<DelegationTokenIdentifier>();
+    dt.decodeFromUrlString(tokenStr);
+    HttpFSServerWebApp.get().get(DelegationTokenManager.class).verifyToken(dt);
+  }
+
+  private void testCancelToken(AuthenticationHandler handler)
+    throws Exception {
+    DelegationTokenOperation op =
+      DelegationTokenOperation.CANCELDELEGATIONTOKEN;
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(request.getParameter(HttpFSFileSystem.OP_PARAM)).
+      thenReturn(op.toString());
+    Mockito.when(request.getMethod()).
+      thenReturn(op.getHttpMethod());
+
+    Assert.assertFalse(handler.managementOperation(null, request, response));
+    Mockito.verify(response).sendError(
+      Mockito.eq(HttpServletResponse.SC_BAD_REQUEST),
+      Mockito.contains("requires the parameter [token]"));
+
+    Mockito.reset(response);
+    Token<DelegationTokenIdentifier> token =
+      HttpFSServerWebApp.get().get(DelegationTokenManager.class).createToken(
+        UserGroupInformation.getCurrentUser(), "foo");
+    Mockito.when(request.getParameter(HttpFSKerberosAuthenticator.TOKEN_PARAM)).
+      thenReturn(token.encodeToUrlString());
+    Assert.assertFalse(handler.managementOperation(null, request, response));
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
+    try {
+      HttpFSServerWebApp.get().get(DelegationTokenManager.class).verifyToken(token);
+      Assert.fail();
+    }
+    catch (DelegationTokenManagerException ex) {
+      Assert.assertTrue(ex.toString().contains("DT01"));
+    }
+  }
+
+  private void testRenewToken(AuthenticationHandler handler)
+    throws Exception {
+    DelegationTokenOperation op =
+      DelegationTokenOperation.RENEWDELEGATIONTOKEN;
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(request.getParameter(HttpFSFileSystem.OP_PARAM)).
+      thenReturn(op.toString());
+    Mockito.when(request.getMethod()).
+      thenReturn(op.getHttpMethod());
+
+    Assert.assertFalse(handler.managementOperation(null, request, response));
+    Mockito.verify(response).sendError(
+      Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED),
+      Mockito.contains("equires SPNEGO authentication established"));
+
+    Mockito.reset(response);
+    AuthenticationToken token = Mockito.mock(AuthenticationToken.class);
+    Mockito.when(token.getUserName()).thenReturn("user");
+    Assert.assertFalse(handler.managementOperation(token, request, response));
+    Mockito.verify(response).sendError(
+      Mockito.eq(HttpServletResponse.SC_BAD_REQUEST),
+      Mockito.contains("requires the parameter [token]"));
+
+    Mockito.reset(response);
+    StringWriter writer = new StringWriter();
+    PrintWriter pwriter = new PrintWriter(writer);
+    Mockito.when(response.getWriter()).thenReturn(pwriter);
+    Token<DelegationTokenIdentifier> dToken =
+      HttpFSServerWebApp.get().get(DelegationTokenManager.class).createToken(
+        UserGroupInformation.getCurrentUser(), "user");
+    Mockito.when(request.getParameter(HttpFSKerberosAuthenticator.TOKEN_PARAM)).
+      thenReturn(dToken.encodeToUrlString());
+    Assert.assertFalse(handler.managementOperation(token, request, response));
+    Mockito.verify(response).setStatus(HttpServletResponse.SC_OK);
+    pwriter.close();
+    Assert.assertTrue(writer.toString().contains("long"));
+    HttpFSServerWebApp.get().get(DelegationTokenManager.class).verifyToken(dToken);
+  }
+
+  @Test
+  @TestDir
+  public void testAuthenticate() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+
+    Configuration httpfsConf = new Configuration(false);
+    HttpFSServerWebApp server =
+      new HttpFSServerWebApp(dir, dir, dir, dir, httpfsConf);
+    server.setAuthority(new InetSocketAddress(InetAddress.getLocalHost(),
+                                              14000));
+    AuthenticationHandler handler =
+      new HttpFSKerberosAuthenticationHandlerForTesting();
+    try {
+      server.init();
+      handler.init(null);
+
+      testValidDelegationToken(handler);
+      testInvalidDelegationToken(handler);
+    } finally {
+      if (handler != null) {
+        handler.destroy();
+      }
+    server.destroy();
+    }
+  }
+
+  private void testValidDelegationToken(AuthenticationHandler handler)
+    throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Token<DelegationTokenIdentifier> dToken =
+      HttpFSServerWebApp.get().get(DelegationTokenManager.class).createToken(
+        UserGroupInformation.getCurrentUser(), "user");
+    Mockito.when(request.getParameter(HttpFSKerberosAuthenticator.DELEGATION_PARAM)).
+      thenReturn(dToken.encodeToUrlString());
+
+    AuthenticationToken token = handler.authenticate(request, response);
+    Assert.assertEquals(UserGroupInformation.getCurrentUser().getShortUserName(),
+                        token.getUserName());
+    Assert.assertEquals(0, token.getExpires());
+    Assert.assertEquals(HttpFSKerberosAuthenticationHandler.TYPE,
+                        token.getType());
+    Assert.assertTrue(token.isExpired());
+  }
+
+  private void testInvalidDelegationToken(AuthenticationHandler handler)
+    throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(request.getParameter(HttpFSKerberosAuthenticator.DELEGATION_PARAM)).
+      thenReturn("invalid");
+
+    try {
+      handler.authenticate(request, response);
+      Assert.fail();
+    } catch (AuthenticationException ex) {
+      //NOP
+    } catch (Exception ex) {
+      Assert.fail();
+    }
+  }
+
+}

+ 111 - 18
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java

@@ -15,11 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.fs.http.server;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.junit.Assert;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -39,9 +37,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
 import org.apache.hadoop.lib.server.Service;
 import org.apache.hadoop.lib.server.ServiceException;
 import org.apache.hadoop.lib.service.Groups;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+import org.apache.hadoop.security.authentication.util.Signer;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.HadoopUsersConfTestHelper;
 import org.apache.hadoop.test.TestDir;
@@ -50,6 +52,8 @@ import org.apache.hadoop.test.TestHdfs;
 import org.apache.hadoop.test.TestHdfsHelper;
 import org.apache.hadoop.test.TestJetty;
 import org.apache.hadoop.test.TestJettyHelper;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
 import org.junit.Test;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.webapp.WebAppContext;
@@ -103,11 +107,13 @@ public class TestHttpFSServer extends HFSTestCase {
     }
 
   }
-  private void createHttpFSServer() throws Exception {
+
+  private void createHttpFSServer(boolean addDelegationTokenAuthHandler)
+    throws Exception {
     File homeDir = TestDirHelper.getTestDir();
-    assertTrue(new File(homeDir, "conf").mkdir());
-    assertTrue(new File(homeDir, "log").mkdir());
-    assertTrue(new File(homeDir, "temp").mkdir());
+    Assert.assertTrue(new File(homeDir, "conf").mkdir());
+    Assert.assertTrue(new File(homeDir, "log").mkdir());
+    Assert.assertTrue(new File(homeDir, "temp").mkdir());
     HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath());
 
     File secretFile = new File(new File(homeDir, "conf"), "secret");
@@ -128,6 +134,10 @@ public class TestHttpFSServer extends HFSTestCase {
 
     //HTTPFS configuration
     conf = new Configuration(false);
+    if (addDelegationTokenAuthHandler) {
+     conf.set("httpfs.authentication.type",
+              HttpFSKerberosAuthenticationHandlerForTesting.class.getName());
+    }
     conf.set("httpfs.services.ext", MockGroups.class.getName());
     conf.set("httpfs.admin.group", HadoopUsersConfTestHelper.
       getHadoopUserGroups(HadoopUsersConfTestHelper.getHadoopUsers()[0])[0]);
@@ -147,6 +157,9 @@ public class TestHttpFSServer extends HFSTestCase {
     Server server = TestJettyHelper.getJettyServer();
     server.addHandler(context);
     server.start();
+    if (addDelegationTokenAuthHandler) {
+      HttpFSServerWebApp.get().setAuthority(TestJettyHelper.getAuthority());
+    }
   }
 
   @Test
@@ -154,28 +167,28 @@ public class TestHttpFSServer extends HFSTestCase {
   @TestJetty
   @TestHdfs
   public void instrumentation() throws Exception {
-    createHttpFSServer();
+    createHttpFSServer(false);
 
     URL url = new URL(TestJettyHelper.getJettyURL(),
                       MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation", "nobody"));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED);
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED);
 
     url = new URL(TestJettyHelper.getJettyURL(),
                   MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation",
                                        HadoopUsersConfTestHelper.getHadoopUsers()[0]));
     conn = (HttpURLConnection) url.openConnection();
-    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
     BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
     String line = reader.readLine();
     reader.close();
-    assertTrue(line.contains("\"counters\":{"));
+    Assert.assertTrue(line.contains("\"counters\":{"));
 
     url = new URL(TestJettyHelper.getJettyURL(),
                   MessageFormat.format("/webhdfs/v1/foo?user.name={0}&op=instrumentation",
                                        HadoopUsersConfTestHelper.getHadoopUsers()[0]));
     conn = (HttpURLConnection) url.openConnection();
-    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
   }
 
   @Test
@@ -183,13 +196,13 @@ public class TestHttpFSServer extends HFSTestCase {
   @TestJetty
   @TestHdfs
   public void testHdfsAccess() throws Exception {
-    createHttpFSServer();
+    createHttpFSServer(false);
 
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     URL url = new URL(TestJettyHelper.getJettyURL(),
                       MessageFormat.format("/webhdfs/v1/?user.name={0}&op=liststatus", user));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
     BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
     reader.readLine();
     reader.close();
@@ -200,7 +213,7 @@ public class TestHttpFSServer extends HFSTestCase {
   @TestJetty
   @TestHdfs
   public void testGlobFilter() throws Exception {
-    createHttpFSServer();
+    createHttpFSServer(false);
 
     FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
     fs.mkdirs(new Path("/tmp"));
@@ -210,7 +223,7 @@ public class TestHttpFSServer extends HFSTestCase {
     URL url = new URL(TestJettyHelper.getJettyURL(),
                       MessageFormat.format("/webhdfs/v1/tmp?user.name={0}&op=liststatus&filter=f*", user));
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
     BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
     reader.readLine();
     reader.close();
@@ -221,7 +234,7 @@ public class TestHttpFSServer extends HFSTestCase {
   @TestJetty
   @TestHdfs
   public void testPutNoOperation() throws Exception {
-    createHttpFSServer();
+    createHttpFSServer(false);
 
     String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
     URL url = new URL(TestJettyHelper.getJettyURL(),
@@ -230,7 +243,87 @@ public class TestHttpFSServer extends HFSTestCase {
     conn.setDoInput(true);
     conn.setDoOutput(true);
     conn.setRequestMethod("PUT");
-    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
+    Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testDelegationTokenOperations() throws Exception {
+    createHttpFSServer(true);
+
+    URL url = new URL(TestJettyHelper.getJettyURL(),
+                      "/webhdfs/v1/?op=GETHOMEDIRECTORY");
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
+                        conn.getResponseCode());
+
+
+    AuthenticationToken token =
+      new AuthenticationToken("u", "p",
+        HttpFSKerberosAuthenticationHandlerForTesting.TYPE);
+    token.setExpires(System.currentTimeMillis() + 100000000);
+    Signer signer = new Signer("secret".getBytes());
+    String tokenSigned = signer.sign(token.toString());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=GETHOMEDIRECTORY");
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestProperty("Cookie",
+                            AuthenticatedURL.AUTH_COOKIE  + "=" + tokenSigned);
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=GETDELEGATIONTOKEN");
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestProperty("Cookie",
+                            AuthenticatedURL.AUTH_COOKIE  + "=" + tokenSigned);
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    JSONObject json = (JSONObject)
+      new JSONParser().parse(new InputStreamReader(conn.getInputStream()));
+    json = (JSONObject)
+      json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON);
+    String tokenStr = (String)
+        json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    conn.setRequestProperty("Cookie",
+                            AuthenticatedURL.AUTH_COOKIE  + "=" + tokenSigned);
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=CANCELDELEGATIONTOKEN&token=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod("PUT");
+    Assert.assertEquals(HttpURLConnection.HTTP_OK,
+                        conn.getResponseCode());
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr);
+    conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED,
+                        conn.getResponseCode());
   }
 
 }

+ 236 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java.orig

@@ -0,0 +1,236 @@
+/**
+ * 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.fs.http.server;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Writer;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.text.MessageFormat;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.lib.server.Service;
+import org.apache.hadoop.lib.server.ServiceException;
+import org.apache.hadoop.lib.service.Groups;
+import org.apache.hadoop.test.HFSTestCase;
+import org.apache.hadoop.test.HadoopUsersConfTestHelper;
+import org.apache.hadoop.test.TestDir;
+import org.apache.hadoop.test.TestDirHelper;
+import org.apache.hadoop.test.TestHdfs;
+import org.apache.hadoop.test.TestHdfsHelper;
+import org.apache.hadoop.test.TestJetty;
+import org.apache.hadoop.test.TestJettyHelper;
+import org.junit.Test;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.webapp.WebAppContext;
+
+public class TestHttpFSServer extends HFSTestCase {
+
+  @Test
+  @TestDir
+  @TestJetty
+  public void server() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+
+    Configuration httpfsConf = new Configuration(false);
+    HttpFSServerWebApp server = new HttpFSServerWebApp(dir, dir, dir, dir, httpfsConf);
+    server.init();
+    server.destroy();
+  }
+
+  public static class MockGroups implements Service,Groups {
+
+    @Override
+    public void init(org.apache.hadoop.lib.server.Server server) throws ServiceException {
+    }
+
+    @Override
+    public void postInit() throws ServiceException {
+    }
+
+    @Override
+    public void destroy() {
+    }
+
+    @Override
+    public Class[] getServiceDependencies() {
+      return new Class[0];
+    }
+
+    @Override
+    public Class getInterface() {
+      return Groups.class;
+    }
+
+    @Override
+    public void serverStatusChange(org.apache.hadoop.lib.server.Server.Status oldStatus,
+                                   org.apache.hadoop.lib.server.Server.Status newStatus) throws ServiceException {
+    }
+
+    @Override
+    public List<String> getGroups(String user) throws IOException {
+      return Arrays.asList(HadoopUsersConfTestHelper.getHadoopUserGroups(user));
+    }
+
+  }
+  private void createHttpFSServer() throws Exception {
+    File homeDir = TestDirHelper.getTestDir();
+    assertTrue(new File(homeDir, "conf").mkdir());
+    assertTrue(new File(homeDir, "log").mkdir());
+    assertTrue(new File(homeDir, "temp").mkdir());
+    HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath());
+
+    File secretFile = new File(new File(homeDir, "conf"), "secret");
+    Writer w = new FileWriter(secretFile);
+    w.write("secret");
+    w.close();
+
+    //HDFS configuration
+    File hadoopConfDir = new File(new File(homeDir, "conf"), "hadoop-conf");
+    hadoopConfDir.mkdirs();
+    String fsDefaultName = TestHdfsHelper.getHdfsConf().get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
+    Configuration conf = new Configuration(false);
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
+    File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    //HTTPFS configuration
+    conf = new Configuration(false);
+    conf.set("httpfs.services.ext", MockGroups.class.getName());
+    conf.set("httpfs.admin.group", HadoopUsersConfTestHelper.
+      getHadoopUserGroups(HadoopUsersConfTestHelper.getHadoopUsers()[0])[0]);
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".groups",
+             HadoopUsersConfTestHelper.getHadoopProxyUserGroups());
+    conf.set("httpfs.proxyuser." + HadoopUsersConfTestHelper.getHadoopProxyUser() + ".hosts",
+             HadoopUsersConfTestHelper.getHadoopProxyUserHosts());
+    conf.set("httpfs.authentication.signature.secret.file", secretFile.getAbsolutePath());
+    File httpfsSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
+    os = new FileOutputStream(httpfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    URL url = cl.getResource("webapp");
+    WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
+    Server server = TestJettyHelper.getJettyServer();
+    server.addHandler(context);
+    server.start();
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void instrumentation() throws Exception {
+    createHttpFSServer();
+
+    URL url = new URL(TestJettyHelper.getJettyURL(),
+                      MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation", "nobody"));
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_UNAUTHORIZED);
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  MessageFormat.format("/webhdfs/v1?user.name={0}&op=instrumentation",
+                                       HadoopUsersConfTestHelper.getHadoopUsers()[0]));
+    conn = (HttpURLConnection) url.openConnection();
+    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+    String line = reader.readLine();
+    reader.close();
+    assertTrue(line.contains("\"counters\":{"));
+
+    url = new URL(TestJettyHelper.getJettyURL(),
+                  MessageFormat.format("/webhdfs/v1/foo?user.name={0}&op=instrumentation",
+                                       HadoopUsersConfTestHelper.getHadoopUsers()[0]));
+    conn = (HttpURLConnection) url.openConnection();
+    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testHdfsAccess() throws Exception {
+    createHttpFSServer();
+
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
+    URL url = new URL(TestJettyHelper.getJettyURL(),
+                      MessageFormat.format("/webhdfs/v1/?user.name={0}&op=liststatus", user));
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+    reader.readLine();
+    reader.close();
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testGlobFilter() throws Exception {
+    createHttpFSServer();
+
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    fs.mkdirs(new Path("/tmp"));
+    fs.create(new Path("/tmp/foo.txt")).close();
+
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
+    URL url = new URL(TestJettyHelper.getJettyURL(),
+                      MessageFormat.format("/webhdfs/v1/tmp?user.name={0}&op=liststatus&filter=f*", user));
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+    BufferedReader reader = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+    reader.readLine();
+    reader.close();
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testPutNoOperation() throws Exception {
+    createHttpFSServer();
+
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
+    URL url = new URL(TestJettyHelper.getJettyURL(),
+                      MessageFormat.format("/webhdfs/v1/foo?user.name={0}", user));
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setDoInput(true);
+    conn.setDoOutput(true);
+    conn.setRequestMethod("PUT");
+    assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_BAD_REQUEST);
+  }
+
+}

+ 291 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java

@@ -0,0 +1,291 @@
+/**
+ * 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.fs.http.server;
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.DelegationTokenRenewer;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.HFSTestCase;
+import org.apache.hadoop.test.KerberosTestUtils;
+import org.apache.hadoop.test.TestDir;
+import org.apache.hadoop.test.TestDirHelper;
+import org.apache.hadoop.test.TestHdfs;
+import org.apache.hadoop.test.TestHdfsHelper;
+import org.apache.hadoop.test.TestJetty;
+import org.apache.hadoop.test.TestJettyHelper;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.junit.After;
+import org.junit.Test;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.webapp.WebAppContext;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Writer;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.security.PrivilegedExceptionAction;
+import java.util.concurrent.Callable;
+
+public class TestHttpFSWithKerberos extends HFSTestCase {
+
+  @After
+  public void resetUGI() {
+    Configuration conf = new Configuration();
+    UserGroupInformation.setConfiguration(conf);
+  }
+
+  private void createHttpFSServer() throws Exception {
+    File homeDir = TestDirHelper.getTestDir();
+    Assert.assertTrue(new File(homeDir, "conf").mkdir());
+    Assert.assertTrue(new File(homeDir, "log").mkdir());
+    Assert.assertTrue(new File(homeDir, "temp").mkdir());
+    HttpFSServerWebApp.setHomeDirForCurrentThread(homeDir.getAbsolutePath());
+
+    File secretFile = new File(new File(homeDir, "conf"), "secret");
+    Writer w = new FileWriter(secretFile);
+    w.write("secret");
+    w.close();
+
+    //HDFS configuration
+    File hadoopConfDir = new File(new File(homeDir, "conf"), "hadoop-conf");
+    hadoopConfDir.mkdirs();
+    String fsDefaultName = TestHdfsHelper.getHdfsConf()
+      .get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY);
+    Configuration conf = new Configuration(false);
+    conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
+    File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
+    OutputStream os = new FileOutputStream(hdfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    conf = new Configuration(false);
+    conf.set("httpfs.proxyuser.client.hosts", "*");
+    conf.set("httpfs.proxyuser.client.groups", "*");
+
+    conf.set("httpfs.authentication.type", "kerberos");
+
+    conf.set("httpfs.authentication.signature.secret.file",
+             secretFile.getAbsolutePath());
+    File httpfsSite = new File(new File(homeDir, "conf"), "httpfs-site.xml");
+    os = new FileOutputStream(httpfsSite);
+    conf.writeXml(os);
+    os.close();
+
+    ClassLoader cl = Thread.currentThread().getContextClassLoader();
+    URL url = cl.getResource("webapp");
+    WebAppContext context = new WebAppContext(url.getPath(), "/webhdfs");
+    Server server = TestJettyHelper.getJettyServer();
+    server.addHandler(context);
+    server.start();
+    HttpFSServerWebApp.get().setAuthority(TestJettyHelper.getAuthority());
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testValidHttpFSAccess() throws Exception {
+    createHttpFSServer();
+
+    KerberosTestUtils.doAsClient(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        URL url = new URL(TestJettyHelper.getJettyURL(),
+                          "/webhdfs/v1/?op=GETHOMEDIRECTORY");
+        AuthenticatedURL aUrl = new AuthenticatedURL();
+        AuthenticatedURL.Token aToken = new AuthenticatedURL.Token();
+        HttpURLConnection conn = aUrl.openConnection(url, aToken);
+        Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+        return null;
+      }
+    });
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testInvalidadHttpFSAccess() throws Exception {
+    createHttpFSServer();
+
+    URL url = new URL(TestJettyHelper.getJettyURL(),
+                      "/webhdfs/v1/?op=GETHOMEDIRECTORY");
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    Assert.assertEquals(conn.getResponseCode(),
+                        HttpURLConnection.HTTP_UNAUTHORIZED);
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testDelegationTokenHttpFSAccess() throws Exception {
+    createHttpFSServer();
+
+    KerberosTestUtils.doAsClient(new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        //get delegation token doing SPNEGO authentication
+        URL url = new URL(TestJettyHelper.getJettyURL(),
+                          "/webhdfs/v1/?op=GETDELEGATIONTOKEN");
+        AuthenticatedURL aUrl = new AuthenticatedURL();
+        AuthenticatedURL.Token aToken = new AuthenticatedURL.Token();
+        HttpURLConnection conn = aUrl.openConnection(url, aToken);
+        Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+        JSONObject json = (JSONObject) new JSONParser()
+          .parse(new InputStreamReader(conn.getInputStream()));
+        json =
+          (JSONObject) json
+            .get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON);
+        String tokenStr = (String) json
+          .get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
+
+        //access httpfs using the delegation token
+        url = new URL(TestJettyHelper.getJettyURL(),
+                      "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" +
+                      tokenStr);
+        conn = (HttpURLConnection) url.openConnection();
+        Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+
+        //try to renew the delegation token without SPNEGO credentials
+        url = new URL(TestJettyHelper.getJettyURL(),
+                      "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr);
+        conn = (HttpURLConnection) url.openConnection();
+        conn.setRequestMethod("PUT");
+        Assert.assertEquals(conn.getResponseCode(),
+                            HttpURLConnection.HTTP_UNAUTHORIZED);
+
+        //renew the delegation token with SPNEGO credentials
+        url = new URL(TestJettyHelper.getJettyURL(),
+                      "/webhdfs/v1/?op=RENEWDELEGATIONTOKEN&token=" + tokenStr);
+        conn = aUrl.openConnection(url, aToken);
+        conn.setRequestMethod("PUT");
+        Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+
+        //cancel delegation token, no need for SPNEGO credentials
+        url = new URL(TestJettyHelper.getJettyURL(),
+                      "/webhdfs/v1/?op=CANCELDELEGATIONTOKEN&token=" +
+                      tokenStr);
+        conn = (HttpURLConnection) url.openConnection();
+        conn.setRequestMethod("PUT");
+        Assert.assertEquals(conn.getResponseCode(), HttpURLConnection.HTTP_OK);
+
+        //try to access httpfs with the canceled delegation token
+        url = new URL(TestJettyHelper.getJettyURL(),
+                      "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" +
+                      tokenStr);
+        conn = (HttpURLConnection) url.openConnection();
+        Assert.assertEquals(conn.getResponseCode(),
+                            HttpURLConnection.HTTP_UNAUTHORIZED);
+        return null;
+      }
+    });
+  }
+
+  @SuppressWarnings("deprecation")
+  private void testDelegationTokenWithFS(Class fileSystemClass)
+    throws Exception {
+    createHttpFSServer();
+    Configuration conf = new Configuration();
+    conf.set("fs.webhdfs.impl", fileSystemClass.getName());
+    conf.set("fs.hdfs.impl.disable.cache", "true");
+    URI uri = new URI( "webhdfs://" +
+                       TestJettyHelper.getJettyURL().toURI().getAuthority());
+    FileSystem fs = FileSystem.get(uri, conf);
+    Token<?> token = fs.getDelegationToken("foo");
+    fs.close();
+    fs = FileSystem.get(uri, conf);
+    ((DelegationTokenRenewer.Renewable) fs).setDelegationToken(token);
+    fs.listStatus(new Path("/"));
+    fs.close();
+  }
+
+  private void testDelegationTokenWithinDoAs(
+    final Class fileSystemClass, boolean proxyUser) throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+    UserGroupInformation.loginUserFromKeytab("client",
+                                             "/Users/tucu/tucu.keytab");
+    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+    if (proxyUser) {
+      ugi = UserGroupInformation.createProxyUser("foo", ugi);
+    }
+    conf = new Configuration();
+    UserGroupInformation.setConfiguration(conf);
+    ugi.doAs(
+      new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          testDelegationTokenWithFS(fileSystemClass);
+          return null;
+        }
+      });
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testDelegationTokenWithHttpFSFileSystem() throws Exception {
+    testDelegationTokenWithinDoAs(HttpFSFileSystem.class, false);
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testDelegationTokenWithWebhdfsFileSystem() throws Exception {
+    testDelegationTokenWithinDoAs(WebHdfsFileSystem.class, false);
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testDelegationTokenWithHttpFSFileSystemProxyUser()
+    throws Exception {
+    testDelegationTokenWithinDoAs(HttpFSFileSystem.class, true);
+  }
+
+  // TODO: WebHdfsFilesystem does work with ProxyUser HDFS-3509
+  //    @Test
+  //    @TestDir
+  //    @TestJetty
+  //    @TestHdfs
+  //    public void testDelegationTokenWithWebhdfsFileSystemProxyUser()
+  //      throws Exception {
+  //      testDelegationTokenWithinDoAs(WebHdfsFileSystem.class, true);
+  //    }
+
+}

+ 83 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestDelegationTokenManagerService.java

@@ -0,0 +1,83 @@
+/**
+ * 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.lib.service.security;
+
+import junit.framework.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
+import org.apache.hadoop.lib.server.Server;
+import org.apache.hadoop.lib.service.DelegationTokenManager;
+import org.apache.hadoop.lib.service.DelegationTokenManagerException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.test.HTestCase;
+import org.apache.hadoop.test.TestDir;
+import org.apache.hadoop.test.TestDirHelper;
+import org.apache.hadoop.util.StringUtils;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+
+public class TestDelegationTokenManagerService extends HTestCase {
+
+  @Test
+  @TestDir
+  public void service() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+    Configuration conf = new Configuration(false);
+    conf.set("server.services", StringUtils.join(",",
+      Arrays.asList(DelegationTokenManagerService.class.getName())));
+    Server server = new Server("server", dir, dir, dir, dir, conf);
+    server.init();
+    DelegationTokenManager tm = server.get(DelegationTokenManager.class);
+    Assert.assertNotNull(tm);
+    server.destroy();
+  }
+
+  @Test
+  @TestDir
+  @SuppressWarnings("unchecked")
+  public void tokens() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+    Configuration conf = new Configuration(false);
+    conf.set("server.services", StringUtils.join(",",
+      Arrays.asList(DelegationTokenManagerService.class.getName())));
+    HttpFSServerWebApp server = new HttpFSServerWebApp(dir, dir, dir, dir, conf);
+    server.setAuthority(new InetSocketAddress(InetAddress.getLocalHost(), 14000));
+    server.init();
+    DelegationTokenManager tm = server.get(DelegationTokenManager.class);
+    Token token = tm.createToken(UserGroupInformation.getCurrentUser(), "foo");
+    Assert.assertNotNull(token);
+    tm.verifyToken(token);
+    Assert.assertTrue(tm.renewToken(token, "foo") > System.currentTimeMillis());
+    tm.cancelToken(token, "foo");
+    try {
+      tm.verifyToken(token);
+      Assert.fail();
+    } catch (DelegationTokenManagerException ex) {
+      //NOP
+    } catch (Exception ex) {
+      Assert.fail();
+    }
+    server.destroy();
+  }
+
+}

+ 138 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/KerberosTestUtils.java

@@ -0,0 +1,138 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.test;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
+
+import java.io.File;
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+
+/**
+ * Test helper class for Java Kerberos setup.
+ */
+public class KerberosTestUtils {
+  private static final String PREFIX = "httpfs.test.";
+
+  public static final String REALM = PREFIX + "kerberos.realm";
+
+  public static final String CLIENT_PRINCIPAL =
+    PREFIX + "kerberos.client.principal";
+
+  public static final String SERVER_PRINCIPAL =
+    PREFIX + "kerberos.server.principal";
+
+  public static final String KEYTAB_FILE = PREFIX + "kerberos.keytab.file";
+
+  public static String getRealm() {
+    return System.getProperty(REALM, "LOCALHOST");
+  }
+
+  public static String getClientPrincipal() {
+    return System.getProperty(CLIENT_PRINCIPAL, "client") + "@" + getRealm();
+  }
+
+  public static String getServerPrincipal() {
+    return System.getProperty(SERVER_PRINCIPAL,
+                              "HTTP/localhost") + "@" + getRealm();
+  }
+
+  public static String getKeytabFile() {
+    String keytabFile =
+      new File(System.getProperty("user.home"),
+               System.getProperty("user.name") + ".keytab").toString();
+    return System.getProperty(KEYTAB_FILE, keytabFile);
+  }
+
+  private static class KerberosConfiguration extends Configuration {
+    private String principal;
+
+    public KerberosConfiguration(String principal) {
+      this.principal = principal;
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      Map<String, String> options = new HashMap<String, String>();
+      options.put("keyTab", KerberosTestUtils.getKeytabFile());
+      options.put("principal", principal);
+      options.put("useKeyTab", "true");
+      options.put("storeKey", "true");
+      options.put("doNotPrompt", "true");
+      options.put("useTicketCache", "true");
+      options.put("renewTGT", "true");
+      options.put("refreshKrb5Config", "true");
+      options.put("isInitiator", "true");
+      String ticketCache = System.getenv("KRB5CCNAME");
+      if (ticketCache != null) {
+        options.put("ticketCache", ticketCache);
+      }
+      options.put("debug", "true");
+
+      return new AppConfigurationEntry[]{
+        new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
+                                  AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                                  options),};
+    }
+  }
+
+  public static <T> T doAs(String principal, final Callable<T> callable)
+    throws Exception {
+    LoginContext loginContext = null;
+    try {
+      Set<Principal> principals = new HashSet<Principal>();
+      principals.add(
+        new KerberosPrincipal(KerberosTestUtils.getClientPrincipal()));
+      Subject subject = new Subject(false, principals, new HashSet<Object>(),
+                                    new HashSet<Object>());
+      loginContext = new LoginContext("", subject, null,
+                                      new KerberosConfiguration(principal));
+      loginContext.login();
+      subject = loginContext.getSubject();
+      return Subject.doAs(subject, new PrivilegedExceptionAction<T>() {
+        @Override
+        public T run() throws Exception {
+          return callable.call();
+        }
+      });
+    } catch (PrivilegedActionException ex) {
+      throw ex.getException();
+    } finally {
+      if (loginContext != null) {
+        loginContext.logout();
+      }
+    }
+  }
+
+  public static <T> T doAsClient(Callable<T> callable) throws Exception {
+    return doAs(getClientPrincipal(), callable);
+  }
+
+  public static <T> T doAsServer(Callable<T> callable) throws Exception {
+    return doAs(getServerPrincipal(), callable);
+  }
+
+}

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestDirHelper.java

@@ -73,7 +73,7 @@ public class TestDirHelper implements MethodRule {
         System.exit(-1);
       }
 
-      TEST_DIR_ROOT = new File(TEST_DIR_ROOT, "testdir").getAbsolutePath();
+      TEST_DIR_ROOT = new File(TEST_DIR_ROOT, "test-dir").getAbsolutePath();
       System.setProperty(TEST_DIR_PROP, TEST_DIR_ROOT);
 
       File dir = new File(TEST_DIR_ROOT);
@@ -83,8 +83,6 @@ public class TestDirHelper implements MethodRule {
         System.exit(-1);
       }
 
-      System.setProperty("test.circus", "true");
-
       System.out.println(">>> " + TEST_DIR_PROP + "        : " + System.getProperty(TEST_DIR_PROP));
     } catch (IOException ex) {
       throw new RuntimeException(ex);

+ 22 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java

@@ -18,9 +18,11 @@
 package org.apache.hadoop.test;
 
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.ServerSocket;
 import java.net.URL;
+import java.net.UnknownHostException;
 
 import org.junit.Test;
 import org.junit.rules.MethodRule;
@@ -65,9 +67,9 @@ public class TestJettyHelper implements MethodRule {
 
   private Server createJettyServer() {
     try {
-
-      String host = InetAddress.getLocalHost().getHostName();
-      ServerSocket ss = new ServerSocket(0);
+      InetAddress localhost = InetAddress.getByName("localhost");
+      String host = "localhost";
+      ServerSocket ss = new ServerSocket(0, 50, localhost);
       int port = ss.getLocalPort();
       ss.close();
       Server server = new Server(0);
@@ -79,6 +81,23 @@ public class TestJettyHelper implements MethodRule {
     }
   }
 
+  /**
+   * Returns the authority (hostname & port) used by the JettyServer.
+   *
+   * @return an <code>InetSocketAddress</code> with the corresponding authority.
+   */
+  public static InetSocketAddress getAuthority() {
+    Server server = getJettyServer();
+    try {
+      InetAddress add =
+        InetAddress.getByName(server.getConnectors()[0].getHost());
+      int port = server.getConnectors()[0].getPort();
+      return new InetSocketAddress(add, port);
+    } catch (UnknownHostException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
   /**
    * Returns a Jetty server ready to be configured and the started. This server
    * is only available when the test method has been annotated with

+ 118 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java.orig

@@ -0,0 +1,118 @@
+/**
+ * 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.test;
+
+import java.net.InetAddress;
+import java.net.MalformedURLException;
+import java.net.ServerSocket;
+import java.net.URL;
+
+import org.junit.Test;
+import org.junit.rules.MethodRule;
+import org.junit.runners.model.FrameworkMethod;
+import org.junit.runners.model.Statement;
+import org.mortbay.jetty.Server;
+
+public class TestJettyHelper implements MethodRule {
+
+  @Test
+  public void dummy() {
+  }
+
+  private static ThreadLocal<Server> TEST_SERVLET_TL = new InheritableThreadLocal<Server>();
+
+  @Override
+  public Statement apply(final Statement statement, final FrameworkMethod frameworkMethod, final Object o) {
+    return new Statement() {
+      @Override
+      public void evaluate() throws Throwable {
+        Server server = null;
+        TestJetty testJetty = frameworkMethod.getAnnotation(TestJetty.class);
+        if (testJetty != null) {
+          server = createJettyServer();
+        }
+        try {
+          TEST_SERVLET_TL.set(server);
+          statement.evaluate();
+        } finally {
+          TEST_SERVLET_TL.remove();
+          if (server != null && server.isRunning()) {
+            try {
+              server.stop();
+            } catch (Exception ex) {
+              throw new RuntimeException("Could not stop embedded servlet container, " + ex.getMessage(), ex);
+            }
+          }
+        }
+      }
+    };
+  }
+
+  private Server createJettyServer() {
+    try {
+
+      String host = InetAddress.getLocalHost().getHostName();
+      ServerSocket ss = new ServerSocket(0);
+      int port = ss.getLocalPort();
+      ss.close();
+      Server server = new Server(0);
+      server.getConnectors()[0].setHost(host);
+      server.getConnectors()[0].setPort(port);
+      return server;
+    } catch (Exception ex) {
+      throw new RuntimeException("Could not stop embedded servlet container, " + ex.getMessage(), ex);
+    }
+  }
+
+  /**
+   * Returns a Jetty server ready to be configured and the started. This server
+   * is only available when the test method has been annotated with
+   * {@link TestJetty}. Refer to {@link HTestCase} header for details.
+   * <p/>
+   * Once configured, the Jetty server should be started. The server will be
+   * automatically stopped when the test method ends.
+   *
+   * @return a Jetty server ready to be configured and the started.
+   */
+  public static Server getJettyServer() {
+    Server server = TEST_SERVLET_TL.get();
+    if (server == null) {
+      throw new IllegalStateException("This test does not use @TestJetty");
+    }
+    return server;
+  }
+
+  /**
+   * Returns the base URL (SCHEMA://HOST:PORT) of the test Jetty server
+   * (see {@link #getJettyServer()}) once started.
+   *
+   * @return the base URL (SCHEMA://HOST:PORT) of the test Jetty server.
+   */
+  public static URL getJettyURL() {
+    Server server = TEST_SERVLET_TL.get();
+    if (server == null) {
+      throw new IllegalStateException("This test does not use @TestJetty");
+    }
+    try {
+      return new URL("http://" + server.getConnectors()[0].getHost() + ":" + server.getConnectors()[0].getPort());
+    } catch (MalformedURLException ex) {
+      throw new RuntimeException("It should never happen, " + ex.getMessage(), ex);
+    }
+  }
+
+}

+ 28 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/resources/krb5.conf

@@ -0,0 +1,28 @@
+#
+# 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.
+#
+[libdefaults]
+	default_realm = ${kerberos.realm}
+	udp_preference_limit = 1
+	extra_addresses = 127.0.0.1
+[realms]
+	${kerberos.realm} = {
+		admin_server = localhost:88
+		kdc = localhost:88
+	}
+[domain_realm]
+	localhost = ${kerberos.realm}

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

@@ -17,6 +17,8 @@ Release 2.0.1-alpha - UNRELEASED
     HDFS-3518. Add a utility method HdfsUtils.isHealthy(uri) for checking if
     the given HDFS is healthy. (szetszwo)
 
+    HDFS-3113. httpfs does not support delegation tokens. (tucu)
+
   IMPROVEMENTS
 
     HDFS-3390. DFSAdmin should print full stack traces of errors when DEBUG