Pārlūkot izejas kodu

HDFS-5703. Add support for HTTPS and swebhdfs to HttpFS. (tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1560504 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 11 gadi atpakaļ
vecāks
revīzija
5e09ae1633
20 mainītis faili ar 479 papildinājumiem un 42 dzēšanām
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
  2. 12 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
  4. 3 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java
  5. 7 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
  6. 34 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpsFSFileSystem.java
  7. 1 1
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java
  8. 8 5
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/DelegationTokenIdentifier.java
  9. 14 5
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/security/DelegationTokenManagerService.java
  10. 9 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/servlet/ServerWebApp.java
  11. 21 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh
  12. 3 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh
  13. 135 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml
  14. 42 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm
  15. 6 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
  16. 99 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java
  17. 27 5
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java
  18. 9 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/TestDelegationTokenManagerService.java
  19. 43 14
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java
  20. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

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

@@ -554,6 +554,9 @@
                     <delete file="${httpfs.tomcat.dist.dir}/conf/server.xml"/>
                     <copy file="${basedir}/src/main/tomcat/server.xml"
                           toDir="${httpfs.tomcat.dist.dir}/conf"/>
+                    <delete file="${httpfs.tomcat.dist.dir}/conf/ssl-server.xml"/>
+                    <copy file="${basedir}/src/main/tomcat/ssl-server.xml"
+                          toDir="${httpfs.tomcat.dist.dir}/conf"/>
                     <delete file="${httpfs.tomcat.dist.dir}/conf/logging.properties"/>
                     <copy file="${basedir}/src/main/tomcat/logging.properties"
                           toDir="${httpfs.tomcat.dist.dir}/conf"/>

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/conf/httpfs-env.sh

@@ -39,3 +39,15 @@
 # The hostname HttpFS server runs on
 #
 # export HTTPFS_HTTP_HOSTNAME=`hostname -f`
+
+# Indicates if HttpFS is using SSL
+#
+# export HTTPFS_SSL_ENABLED=false
+
+# The location of the SSL keystore if using SSL
+#
+# export HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+
+# The password of the SSL keystore if using SSL
+#
+# export HTTPFS_SSL_KEYSTORE_PASS=password

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

@@ -243,7 +243,7 @@ public class HttpFSFileSystem extends FileSystem
     if (makeQualified) {
       path = makeQualified(path);
     }
-    final URL url = HttpFSUtils.createHttpURL(path, params);
+    final URL url = HttpFSUtils.createURL(path, params);
     return doAsRealUserIfNecessary(new Callable<HttpURLConnection>() {
       @Override
       public HttpURLConnection call() throws Exception {

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

@@ -123,7 +123,7 @@ public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
     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);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {
@@ -150,7 +150,7 @@ public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
     params.put(OP_PARAM,
                DelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
     params.put(TOKEN_PARAM, dToken.encodeToUrlString());
-    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {
@@ -172,7 +172,7 @@ public class HttpFSKerberosAuthenticator extends KerberosAuthenticator {
     params.put(OP_PARAM,
                DelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
     params.put(TOKEN_PARAM, dToken.encodeToUrlString());
-    URL url = HttpFSUtils.createHttpURL(new Path(fsURI), params);
+    URL url = HttpFSUtils.createURL(new Path(fsURI), params);
     AuthenticatedURL aUrl =
       new AuthenticatedURL(new HttpFSKerberosAuthenticator());
     try {

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

@@ -55,17 +55,21 @@ public class HttpFSUtils {
    *
    * @return a <code>URL</code> for the HttpFSServer server,
    *
-   * @throws IOException thrown if an IO error occurrs.
+   * @throws IOException thrown if an IO error occurs.
    */
-  static URL createHttpURL(Path path, Map<String, String> params)
+  static URL createURL(Path path, Map<String, String> params)
     throws IOException {
     URI uri = path.toUri();
     String realScheme;
     if (uri.getScheme().equalsIgnoreCase(HttpFSFileSystem.SCHEME)) {
       realScheme = "http";
+    } else if (uri.getScheme().equalsIgnoreCase(HttpsFSFileSystem.SCHEME)) {
+      realScheme = "https";
+
     } else {
       throw new IllegalArgumentException(MessageFormat.format(
-        "Invalid scheme [{0}] it should be 'webhdfs'", uri));
+        "Invalid scheme [{0}] it should be '" + HttpFSFileSystem.SCHEME + "' " +
+            "or '" + HttpsFSFileSystem.SCHEME + "'", uri));
     }
     StringBuilder sb = new StringBuilder();
     sb.append(realScheme).append("://").append(uri.getAuthority()).

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

@@ -0,0 +1,34 @@
+/**
+ * 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;
+
+/**
+ * HttpFSServer implementation of the FileSystemAccess FileSystem for SSL.
+ * <p/>
+ * This implementation allows a user to access HDFS over HTTPS via a
+ * HttpFSServer server.
+ */
+public class HttpsFSFileSystem extends HttpFSFileSystem {
+
+  public static final String SCHEME = "swebhdfs";
+
+  @Override
+  public String getScheme() {
+    return SCHEME;
+  }
+}

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

@@ -94,11 +94,11 @@ public class HttpFSServerWebApp extends ServerWebApp {
    */
   @Override
   public void init() throws ServerException {
-    super.init();
     if (SERVER != null) {
       throw new RuntimeException("HttpFSServer server already initialized");
     }
     SERVER = this;
+    super.init();
     adminGroup = getConfig().get(getPrefixedName(CONF_ADMIN_GROUP), "admin");
     LOG.info("Connects to Namenode [{}]",
              get().get(FileSystemAccess.class).getFileSystemConfiguration().

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

@@ -29,30 +29,33 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
 public class DelegationTokenIdentifier
   extends AbstractDelegationTokenIdentifier {
 
-  public static final Text KIND_NAME = WebHdfsFileSystem.TOKEN_KIND;
+  private Text kind = WebHdfsFileSystem.TOKEN_KIND;
 
-  public DelegationTokenIdentifier() {
+  public DelegationTokenIdentifier(Text kind) {
+    this.kind = kind;
   }
 
   /**
    * Create a new delegation token identifier
    *
+   * @param kind token kind
    * @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) {
+  public DelegationTokenIdentifier(Text kind, Text owner, Text renewer,
+      Text realUser) {
     super(owner, renewer, realUser);
+    this.kind = kind;
   }
 
-
   /**
    * Returns the kind, <code>TOKEN_KIND</code>.
    * @return returns <code>TOKEN_KIND</code>.
    */
   @Override
   public Text getKind() {
-    return KIND_NAME;
+    return kind;
   }
 
 }

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

@@ -19,6 +19,8 @@ package org.apache.hadoop.lib.service.security;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.http.server.HttpFSServerWebApp;
+import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.lib.server.BaseService;
 import org.apache.hadoop.lib.server.ServerException;
@@ -55,6 +57,8 @@ public class DelegationTokenManagerService extends BaseService
 
   DelegationTokenSecretManager secretManager = null;
 
+  private Text tokenKind;
+
   public DelegationTokenManagerService() {
     super(PREFIX);
   }
@@ -70,7 +74,9 @@ public class DelegationTokenManagerService extends BaseService
     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,
+    tokenKind = (HttpFSServerWebApp.get().isSslEnabled())
+                ? SWebHdfsFileSystem.TOKEN_KIND : WebHdfsFileSystem.TOKEN_KIND;
+    secretManager = new DelegationTokenSecretManager(tokenKind, updateInterval,
                                                      maxLifetime,
                                                      renewInterval, HOUR);
     try {
@@ -122,7 +128,7 @@ public class DelegationTokenManagerService extends BaseService
       realUser = new Text(ugi.getRealUser().getUserName());
     }
     DelegationTokenIdentifier tokenIdentifier =
-      new DelegationTokenIdentifier(owner, new Text(renewer), realUser);
+      new DelegationTokenIdentifier(tokenKind, owner, new Text(renewer), realUser);
     Token<DelegationTokenIdentifier> token =
       new Token<DelegationTokenIdentifier>(tokenIdentifier, secretManager);
     try {
@@ -188,7 +194,7 @@ public class DelegationTokenManagerService extends BaseService
     throws DelegationTokenManagerException {
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream dis = new DataInputStream(buf);
-    DelegationTokenIdentifier id = new DelegationTokenIdentifier();
+    DelegationTokenIdentifier id = new DelegationTokenIdentifier(tokenKind);
     try {
       id.readFields(dis);
       dis.close();
@@ -203,6 +209,8 @@ public class DelegationTokenManagerService extends BaseService
   private static class DelegationTokenSecretManager
     extends AbstractDelegationTokenSecretManager<DelegationTokenIdentifier> {
 
+    private Text tokenKind;
+
     /**
      * Create a secret manager
      *
@@ -215,17 +223,18 @@ public class DelegationTokenManagerService extends BaseService
      * scanned
      * for expired tokens
      */
-    public DelegationTokenSecretManager(long delegationKeyUpdateInterval,
+    public DelegationTokenSecretManager(Text tokenKind, long delegationKeyUpdateInterval,
                                         long delegationTokenMaxLifetime,
                                         long delegationTokenRenewInterval,
                                         long delegationTokenRemoverScanInterval) {
       super(delegationKeyUpdateInterval, delegationTokenMaxLifetime,
             delegationTokenRenewInterval, delegationTokenRemoverScanInterval);
+      this.tokenKind = tokenKind;
     }
 
     @Override
     public DelegationTokenIdentifier createIdentifier() {
-      return new DelegationTokenIdentifier();
+      return new DelegationTokenIdentifier(tokenKind);
     }
 
   }

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

@@ -44,6 +44,7 @@ public abstract class ServerWebApp extends Server implements ServletContextListe
   private static final String TEMP_DIR = ".temp.dir";
   private static final String HTTP_HOSTNAME = ".http.hostname";
   private static final String HTTP_PORT = ".http.port";
+  public static final String SSL_ENABLED = ".ssl.enabled";
 
   private static ThreadLocal<String> HOME_DIR_TL = new ThreadLocal<String>();
 
@@ -225,4 +226,12 @@ public abstract class ServerWebApp extends Server implements ServletContextListe
   public void setAuthority(InetSocketAddress authority) {
     this.authority = authority;
   }
+
+
+  /**
+   *
+   */
+  public boolean isSslEnabled() {
+    return Boolean.valueOf(System.getProperty(getName() + SSL_ENABLED, "false"));
+  }
 }

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/libexec/httpfs-config.sh

@@ -143,6 +143,27 @@ else
   print "Using   HTTPFS_HTTP_HOSTNAME: ${HTTPFS_HTTP_HOSTNAME}"
 fi
 
+if [ "${HTTPFS_SSL_ENABLED}" = "" ]; then
+  export HTTPFS_SSL_ENABLED="false"
+  print "Setting HTTPFS_SSL_ENABLED: ${HTTPFS_SSL_ENABLED}"
+else
+  print "Using   HTTPFS_SSL_ENABLED: ${HTTPFS_SSL_ENABLED}"
+fi
+
+if [ "${HTTPFS_SSL_KEYSTORE_FILE}" = "" ]; then
+  export HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+  print "Setting HTTPFS_SSL_KEYSTORE_FILE:     ${HTTPFS_SSL_KEYSTORE_FILE}"
+else
+  print "Using   HTTPFS_SSL_KEYSTORE_FILE:     ${HTTPFS_SSL_KEYSTORE_FILE}"
+fi
+
+if [ "${HTTPFS_SSL_KEYSTORE_PASS}" = "" ]; then
+  export HTTPFS_SSL_KEYSTORE_PASS=password
+  print "Setting HTTPFS_SSL_KEYSTORE_PASS:     ${HTTPFS_SSL_KEYSTORE_PASS}"
+else
+  print "Using   HTTPFS_SSL_KEYSTORE_PASS:     ${HTTPFS_SSL_KEYSTORE_PASS}"
+fi
+
 if [ "${CATALINA_BASE}" = "" ]; then
   export CATALINA_BASE=${HTTPFS_HOME}/share/hadoop/httpfs/tomcat
   print "Setting CATALINA_BASE:       ${CATALINA_BASE}"

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/sbin/httpfs.sh

@@ -43,6 +43,9 @@ catalina_opts="${catalina_opts} -Dhttpfs.temp.dir=${HTTPFS_TEMP}";
 catalina_opts="${catalina_opts} -Dhttpfs.admin.port=${HTTPFS_ADMIN_PORT}";
 catalina_opts="${catalina_opts} -Dhttpfs.http.port=${HTTPFS_HTTP_PORT}";
 catalina_opts="${catalina_opts} -Dhttpfs.http.hostname=${HTTPFS_HTTP_HOSTNAME}";
+catalina_opts="${catalina_opts} -Dhttpfs.ssl.enabled=${HTTPFS_SSL_ENABLED}";
+catalina_opts="${catalina_opts} -Dhttpfs.ssl.keystore.file=${HTTPFS_SSL_KEYSTORE_FILE}";
+catalina_opts="${catalina_opts} -Dhttpfs.ssl.keystore.pass=${HTTPFS_SSL_KEYSTORE_PASS}";
 
 print "Adding to CATALINA_OPTS:     ${catalina_opts}"
 

+ 135 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/tomcat/ssl-server.xml

@@ -0,0 +1,135 @@
+<?xml version='1.0' encoding='utf-8'?>
+<!--
+
+   All Rights Reserved.
+
+  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.
+-->
+<!-- Note:  A "Server" is not itself a "Container", so you may not
+     define subcomponents such as "Valves" at this level.
+     Documentation at /docs/config/server.html
+ -->
+<Server port="${httpfs.admin.port}" shutdown="SHUTDOWN">
+
+  <!--APR library loader. Documentation at /docs/apr.html -->
+  <Listener className="org.apache.catalina.core.AprLifecycleListener"
+            SSLEngine="on"/>
+  <!--Initialize Jasper prior to webapps are loaded. Documentation at /docs/jasper-howto.html -->
+  <Listener className="org.apache.catalina.core.JasperListener"/>
+  <!-- Prevent memory leaks due to use of particular java/javax APIs-->
+  <Listener
+    className="org.apache.catalina.core.JreMemoryLeakPreventionListener"/>
+  <!-- JMX Support for the Tomcat server. Documentation at /docs/non-existent.html -->
+  <Listener className="org.apache.catalina.mbeans.ServerLifecycleListener"/>
+  <Listener
+    className="org.apache.catalina.mbeans.GlobalResourcesLifecycleListener"/>
+
+  <!-- Global JNDI resources
+       Documentation at /docs/jndi-resources-howto.html
+  -->
+  <GlobalNamingResources>
+    <!-- Editable user database that can also be used by
+         UserDatabaseRealm to authenticate users
+    -->
+    <Resource name="UserDatabase" auth="Container"
+              type="org.apache.catalina.UserDatabase"
+              description="User database that can be updated and saved"
+              factory="org.apache.catalina.users.MemoryUserDatabaseFactory"
+              pathname="conf/tomcat-users.xml"/>
+  </GlobalNamingResources>
+
+  <!-- A "Service" is a collection of one or more "Connectors" that share
+       a single "Container" Note:  A "Service" is not itself a "Container",
+       so you may not define subcomponents such as "Valves" at this level.
+       Documentation at /docs/config/service.html
+   -->
+  <Service name="Catalina">
+
+    <!--The connectors can use a shared executor, you can define one or more named thread pools-->
+    <!--
+    <Executor name="tomcatThreadPool" namePrefix="catalina-exec-"
+        maxThreads="150" minSpareThreads="4"/>
+    -->
+
+    <!-- Define a SSL HTTP/1.1 Connector on port 8443
+         This connector uses the JSSE configuration, when using APR, the
+         connector should be using the OpenSSL style configuration
+         described in the APR documentation -->
+    <Connector port="${httpfs.http.port}" protocol="HTTP/1.1" SSLEnabled="true"
+               maxThreads="150" scheme="https" secure="true"
+               clientAuth="false" sslProtocol="TLS"
+               keystoreFile="${httpfs.ssl.keystore.file}"
+               keystorePass="${httpfs.ssl.keystore.pass}"/>
+
+    <!-- Define an AJP 1.3 Connector on port 8009 -->
+
+
+    <!-- An Engine represents the entry point (within Catalina) that processes
+ every request.  The Engine implementation for Tomcat stand alone
+ analyzes the HTTP headers included with the request, and passes them
+ on to the appropriate Host (virtual host).
+ Documentation at /docs/config/engine.html -->
+
+    <!-- You should set jvmRoute to support load-balancing via AJP ie :
+    <Engine name="Catalina" defaultHost="localhost" jvmRoute="jvm1">
+    -->
+    <Engine name="Catalina" defaultHost="localhost">
+
+      <!--For clustering, please take a look at documentation at:
+          /docs/cluster-howto.html  (simple how to)
+          /docs/config/cluster.html (reference documentation) -->
+      <!--
+      <Cluster className="org.apache.catalina.ha.tcp.SimpleTcpCluster"/>
+      -->
+
+      <!-- The request dumper valve dumps useful debugging information about
+           the request and response data received and sent by Tomcat.
+           Documentation at: /docs/config/valve.html -->
+      <!--
+      <Valve className="org.apache.catalina.valves.RequestDumperValve"/>
+      -->
+
+      <!-- This Realm uses the UserDatabase configured in the global JNDI
+           resources under the key "UserDatabase".  Any edits
+           that are performed against this UserDatabase are immediately
+           available for use by the Realm.  -->
+      <Realm className="org.apache.catalina.realm.UserDatabaseRealm"
+             resourceName="UserDatabase"/>
+
+      <!-- Define the default virtual host
+           Note: XML Schema validation will not work with Xerces 2.2.
+       -->
+      <Host name="localhost" appBase="webapps"
+            unpackWARs="true" autoDeploy="true"
+            xmlValidation="false" xmlNamespaceAware="false">
+
+        <!-- SingleSignOn valve, share authentication between web applications
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.authenticator.SingleSignOn" />
+        -->
+
+        <!-- Access log processes all example.
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.valves.AccessLogValve" directory="logs"
+               prefix="localhost_access_log." suffix=".txt" pattern="common" resolveHosts="false"/>
+        -->
+
+      </Host>
+    </Engine>
+  </Service>
+</Server>

+ 42 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/site/apt/ServerSetup.apt.vm

@@ -118,4 +118,46 @@ Transfer-Encoding: chunked
   HttpFS supports the following {{{./httpfs-default.html}configuration properties}}
   in the HttpFS's <<<conf/httpfs-site.xml>>> configuration file.
 
+* HttpFS over HTTPS (SSL)
+
+  To configure HttpFS to work over SSL edit the {{httpfs-env.sh}} script in the
+  configuration directory setting the {{HTTPFS_SSL_ENABLED}} to {{true}}.
+
+  In addition, the following 2 properties may be defined (shown with default
+  values):
+
+    * HTTPFS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+
+    * HTTPFS_SSL_KEYSTORE_PASS=password
+
+  In the HttpFS <<<tomcat/conf>>> directory, replace the <<<server.xml>>> file
+  with the  <<<ssl-server.xml>>> file.
+
+
+  You need to create an SSL certificate for the HttpFS server. As the
+  <<<httpfs>>> Unix user, using the Java <<<keytool>>> command to create the
+  SSL certificate:
+
++---+
+$ keytool -genkey -alias tomcat -keyalg RSA
++---+
+
+  You will be asked a series of questions in an interactive prompt.  It will
+  create the keystore file, which will be named <<.keystore>> and located in the
+  <<<httpfs>>> user home directory.
+
+  The password you enter for "keystore password" must match the  value of the
+  <<<HTTPFS_SSL_KEYSTORE_PASS>>> environment variable set in the
+  <<<httpfs-env.sh>>> script in the configuration directory.
+
+  The answer to "What is your first and last name?" (i.e. "CN") must be the
+  hostname of the machine where the HttpFS Server will be running.
+
+  Start HttpFS. It should work over HTTPS.
+
+  Using the Hadoop <<<FileSystem>>> API or the Hadoop FS shell, use the
+  <<<swebhdfs://>>> scheme. Make sure the JVM is picking up the truststore
+  containing the public key of the SSL certificate if using a self-signed
+  certificate.
+
   \[ {{{./index.html}Go Back}} \]

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

@@ -116,10 +116,14 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     return HttpFSFileSystem.class;
   }
 
+  protected String getScheme() {
+    return "webhdfs";
+  }
+
   protected FileSystem getHttpFSFileSystem() throws Exception {
     Configuration conf = new Configuration();
     conf.set("fs.webhdfs.impl", getFileSystemClass().getName());
-    URI uri = new URI("webhdfs://" +
+    URI uri = new URI(getScheme() + "://" +
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
     return FileSystem.get(uri, conf);
   }
@@ -127,7 +131,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
   protected void testGet() throws Exception {
     FileSystem fs = getHttpFSFileSystem();
     Assert.assertNotNull(fs);
-    URI uri = new URI("webhdfs://" +
+    URI uri = new URI(getScheme() + "://" +
                       TestJettyHelper.getJettyURL().toURI().getAuthority());
     Assert.assertEquals(fs.getUri(), uri);
     fs.close();

+ 99 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFWithSWebhdfsFileSystem.java

@@ -0,0 +1,99 @@
+/**
+ * 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.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.TestJettyHelper;
+import org.junit.AfterClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.net.URI;
+import java.net.URL;
+import java.util.UUID;
+
+@RunWith(value = Parameterized.class)
+public class TestHttpFSFWithSWebhdfsFileSystem
+  extends TestHttpFSWithHttpFSFileSystem {
+  private static String classpathDir;
+  private static final String BASEDIR = System.getProperty("test.build.dir",
+      "target/test-dir") + "/" + UUID.randomUUID();
+
+  private static Configuration sslConf;
+
+  {
+    URL url = Thread.currentThread().getContextClassLoader().
+        getResource("classutils.txt");
+    classpathDir = url.toExternalForm();
+    if (classpathDir.startsWith("file:")) {
+      classpathDir = classpathDir.substring("file:".length());
+      classpathDir = classpathDir.substring(0,
+          classpathDir.length() - "/classutils.txt".length());
+    } else {
+      throw new RuntimeException("Cannot find test classes dir");
+    }
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+    String keyStoreDir = new File(BASEDIR).getAbsolutePath();
+    try {
+      sslConf = new Configuration();
+      KeyStoreTestUtil.setupSSLConfig(keyStoreDir, classpathDir, sslConf, false);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    jettyTestHelper = new TestJettyHelper("jks", keyStoreDir + "/serverKS.jks",
+        "serverP");
+  }
+
+  @AfterClass
+  public static void cleanUp() {
+    new File(classpathDir, "ssl-client.xml").delete();
+    new File(classpathDir, "ssl-server.xml").delete();
+  }
+
+  public TestHttpFSFWithSWebhdfsFileSystem(Operation operation) {
+    super(operation);
+  }
+
+  @Override
+  protected Class getFileSystemClass() {
+    return SWebHdfsFileSystem.class;
+  }
+
+  @Override
+  protected String getScheme() {
+    return "swebhdfs";
+  }
+
+  @Override
+  protected FileSystem getHttpFSFileSystem() throws Exception {
+    Configuration conf = new Configuration(sslConf);
+    conf.set("fs.swebhdfs.impl", getFileSystemClass().getName());
+    URI uri = new URI("swebhdfs://" +
+        TestJettyHelper.getJettyURL().toURI().getAuthority());
+    return FileSystem.get(uri, conf);
+  }
+
+}

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

@@ -22,9 +22,13 @@ 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.hdfs.web.SWebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.io.Text;
 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.lib.servlet.ServerWebApp;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.server.AuthenticationHandler;
@@ -51,7 +55,24 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
 
   @Test
   @TestDir
-  public void testManagementOperations() throws Exception {
+  public void testManagementOperationsWebHdfsFileSystem() throws Exception {
+    testManagementOperations(WebHdfsFileSystem.TOKEN_KIND);
+  }
+
+  @Test
+  @TestDir
+  public void testManagementOperationsSWebHdfsFileSystem() throws Exception {
+    try {
+      System.setProperty(HttpFSServerWebApp.NAME +
+          ServerWebApp.SSL_ENABLED, "true");
+      testManagementOperations(SWebHdfsFileSystem.TOKEN_KIND);
+    } finally {
+      System.getProperties().remove(HttpFSServerWebApp.NAME +
+          ServerWebApp.SSL_ENABLED);
+    }
+  }
+
+  private void testManagementOperations(Text expectedTokenKind) throws Exception {
     String dir = TestDirHelper.getTestDir().getAbsolutePath();
 
     Configuration httpfsConf = new Configuration(false);
@@ -67,8 +88,8 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
 
       testNonManagementOperation(handler);
       testManagementOperationErrors(handler);
-      testGetToken(handler, null);
-      testGetToken(handler, "foo");
+      testGetToken(handler, null, expectedTokenKind);
+      testGetToken(handler, "foo", expectedTokenKind);
       testCancelToken(handler);
       testRenewToken(handler);
 
@@ -112,8 +133,8 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
       Mockito.contains("requires SPNEGO"));
   }
 
-  private void testGetToken(AuthenticationHandler handler, String renewer)
-    throws Exception {
+  private void testGetToken(AuthenticationHandler handler, String renewer,
+      Text expectedTokenKind) throws Exception {
     DelegationTokenOperation op = DelegationTokenOperation.GETDELEGATIONTOKEN;
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
@@ -154,6 +175,7 @@ public class TestHttpFSKerberosAuthenticationHandler extends HFSTestCase {
     Token<DelegationTokenIdentifier> dt = new Token<DelegationTokenIdentifier>();
     dt.decodeFromUrlString(tokenStr);
     HttpFSServerWebApp.get().get(DelegationTokenManager.class).verifyToken(dt);
+    Assert.assertEquals(expectedTokenKind, dt.getKind());
   }
 
   private void testCancelToken(AuthenticationHandler handler)

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

@@ -23,6 +23,9 @@ 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.lib.service.hadoop.FileSystemAccessService;
+import org.apache.hadoop.lib.service.instrumentation.InstrumentationService;
+import org.apache.hadoop.lib.service.scheduler.SchedulerService;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.HTestCase;
@@ -43,9 +46,12 @@ public class TestDelegationTokenManagerService extends HTestCase {
   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);
+    conf.set("httpfs.services", StringUtils.join(",",
+      Arrays.asList(InstrumentationService.class.getName(),
+          SchedulerService.class.getName(),
+          FileSystemAccessService.class.getName(),
+          DelegationTokenManagerService.class.getName())));
+    Server server = new HttpFSServerWebApp(dir, dir, dir, dir, conf);
     server.init();
     DelegationTokenManager tm = server.get(DelegationTokenManager.class);
     Assert.assertNotNull(tm);

+ 43 - 14
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestJettyHelper.java

@@ -28,31 +28,46 @@ 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.Connector;
 import org.mortbay.jetty.Server;
+import org.mortbay.jetty.security.SslSocketConnector;
 
 public class TestJettyHelper implements MethodRule {
+  private boolean ssl;
+  private String keyStoreType;
+  private String keyStore;
+  private String keyStorePassword;
+  private Server server;
 
-  @Test
-  public void dummy() {
+  public TestJettyHelper() {
+    this.ssl = false;
   }
 
-  private static ThreadLocal<Server> TEST_SERVLET_TL = new InheritableThreadLocal<Server>();
+  public TestJettyHelper(String keyStoreType, String keyStore,
+      String keyStorePassword) {
+    ssl = true;
+    this.keyStoreType = keyStoreType;
+    this.keyStore = keyStore;
+    this.keyStorePassword = keyStorePassword;
+  }
+
+  private static ThreadLocal<TestJettyHelper> TEST_JETTY_TL =
+      new InheritableThreadLocal<TestJettyHelper>();
 
   @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);
+          TEST_JETTY_TL.set(TestJettyHelper.this);
           statement.evaluate();
         } finally {
-          TEST_SERVLET_TL.remove();
+          TEST_JETTY_TL.remove();
           if (server != null && server.isRunning()) {
             try {
               server.stop();
@@ -73,8 +88,19 @@ public class TestJettyHelper implements MethodRule {
       int port = ss.getLocalPort();
       ss.close();
       Server server = new Server(0);
-      server.getConnectors()[0].setHost(host);
-      server.getConnectors()[0].setPort(port);
+      if (!ssl) {
+        server.getConnectors()[0].setHost(host);
+        server.getConnectors()[0].setPort(port);
+      } else {
+        SslSocketConnector c = new SslSocketConnector();
+        c.setHost(host);
+        c.setPort(port);
+        c.setNeedClientAuth(false);
+        c.setKeystore(keyStore);
+        c.setKeystoreType(keyStoreType);
+        c.setKeyPassword(keyStorePassword);
+        server.setConnectors(new Connector[] {c});
+      }
       return server;
     } catch (Exception ex) {
       throw new RuntimeException("Could not start embedded servlet container, " + ex.getMessage(), ex);
@@ -109,11 +135,11 @@ public class TestJettyHelper implements MethodRule {
    * @return a Jetty server ready to be configured and the started.
    */
   public static Server getJettyServer() {
-    Server server = TEST_SERVLET_TL.get();
-    if (server == null) {
+    TestJettyHelper helper = TEST_JETTY_TL.get();
+    if (helper == null || helper.server == null) {
       throw new IllegalStateException("This test does not use @TestJetty");
     }
-    return server;
+    return helper.server;
   }
 
   /**
@@ -123,12 +149,15 @@ public class TestJettyHelper implements MethodRule {
    * @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) {
+    TestJettyHelper helper = TEST_JETTY_TL.get();
+    if (helper == null || helper.server == null) {
       throw new IllegalStateException("This test does not use @TestJetty");
     }
     try {
-      return new URL("http://" + server.getConnectors()[0].getHost() + ":" + server.getConnectors()[0].getPort());
+      String scheme = (helper.ssl) ? "https" : "http";
+      return new URL(scheme + "://" +
+          helper.server.getConnectors()[0].getHost() + ":" +
+          helper.server.getConnectors()[0].getPort());
     } catch (MalformedURLException ex) {
       throw new RuntimeException("It should never happen, " + ex.getMessage(), ex);
     }

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

@@ -524,6 +524,8 @@ Release 2.4.0 - UNRELEASED
     HDFS-5784. reserve space in edit log header and fsimage header for feature
     flag section (cmccabe)
 
+    HDFS-5703. Add support for HTTPS and swebhdfs to HttpFS. (tucu)
+
   IMPROVEMENTS
 
     HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)