You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by tu...@apache.org on 2012/08/10 00:52:57 UTC

svn commit: r1371525 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/http/ src/main/java/org/apache/hadoop/security/ src/main/resources/ src/test/java/org/apache/hado...

Author: tucu
Date: Thu Aug  9 22:52:56 2012
New Revision: 1371525

URL: http://svn.apache.org/viewvc?rev=1371525&view=rev
Log:
HADOOP-8681. add support for HTTPS to the web UIs. (tucu)

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1371525&r1=1371524&r2=1371525&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt Thu Aug  9 22:52:56 2012
@@ -213,6 +213,8 @@ Branch-2 ( Unreleased changes )
 
     HADOOP-8644. AuthenticatedURL should be able to use SSLFactory. (tucu)
 
+    HADOOP-8681. add support for HTTPS to the web UIs. (tucu)
+
   IMPROVEMENTS
 
     HADOOP-8340. SNAPSHOT build versions should compare as less than their eventual

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java?rev=1371525&r1=1371524&r2=1371525&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java Thu Aug  9 22:52:56 2012
@@ -239,5 +239,8 @@ public class CommonConfigurationKeysPubl
   public static final String  HADOOP_SECURITY_AUTH_TO_LOCAL =
     "hadoop.security.auth_to_local";
 
+  public static final String HADOOP_SSL_ENABLED_KEY = "hadoop.ssl.enabled";
+  public static final boolean HADOOP_SSL_ENABLED_DEFAULT = false;
+
 }
 

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java?rev=1371525&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpConfig.java Thu Aug  9 22:52:56 2012
@@ -0,0 +1,48 @@
+/**
+ * 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.http;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+
+/**
+ * Singleton to get access to Http related configuration.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HttpConfig {
+  private static boolean sslEnabled;
+
+  static {
+    Configuration conf = new Configuration();
+    sslEnabled = conf.getBoolean(
+        CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_KEY,
+        CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_DEFAULT);
+  }
+
+  public static boolean isSecure() {
+    return sslEnabled;
+  }
+
+  public static String getSchemePrefix() {
+    return (isSecure()) ? "https://" : "http://";
+  }
+
+}

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java?rev=1371525&r1=1371524&r2=1371525&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java Thu Aug  9 22:52:56 2012
@@ -24,12 +24,14 @@ import java.io.InterruptedIOException;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.URL;
+import java.security.GeneralSecurityException;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import javax.net.ssl.SSLServerSocketFactory;
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
@@ -56,6 +58,7 @@ import org.apache.hadoop.security.Securi
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.mortbay.io.Buffer;
 import org.mortbay.jetty.Connector;
@@ -105,6 +108,7 @@ public class HttpServer implements Filte
 
   private AccessControlList adminsAcl;
 
+  private SSLFactory sslFactory;
   protected final Server webServer;
   protected final Connector listener;
   protected final WebAppContext webAppContext;
@@ -208,7 +212,23 @@ public class HttpServer implements Filte
     
     if(connector == null) {
       listenerStartedExternally = false;
-      listener = createBaseListener(conf);
+      if (HttpConfig.isSecure()) {
+        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+        try {
+          sslFactory.init();
+        } catch (GeneralSecurityException ex) {
+          throw new IOException(ex);
+        }
+        SslSocketConnector sslListener = new SslSocketConnector() {
+          @Override
+          protected SSLServerSocketFactory createFactory() throws Exception {
+            return sslFactory.createSSLServerSocketFactory();
+          }
+        };
+        listener = sslListener;
+      } else {
+        listener = createBaseListener(conf);
+      }
       listener.setHost(bindAddress);
       listener.setPort(port);
     } else {
@@ -721,6 +741,16 @@ public class HttpServer implements Filte
     }
 
     try {
+      if (sslFactory != null) {
+          sslFactory.destroy();
+      }
+    } catch (Exception e) {
+      LOG.error("Error while destroying the SSLFactory"
+          + webAppContext.getDisplayName(), e);
+      exception = addMultiException(exception, e);
+    }
+
+    try {
       // clear & stop webAppContext attributes to avoid memory leaks.
       webAppContext.clearAttributes();
       webAppContext.stop();

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java?rev=1371525&r1=1371524&r2=1371525&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java Thu Aug  9 22:52:56 2012
@@ -40,10 +40,12 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
 
@@ -65,12 +67,23 @@ public class SecurityUtil {
   static boolean useIpForTokenService;
   @VisibleForTesting
   static HostResolver hostResolver;
-  
+
+  private static SSLFactory sslFactory;
+
   static {
-    boolean useIp = new Configuration().getBoolean(
+    Configuration conf = new Configuration();
+    boolean useIp = conf.getBoolean(
       CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP,
       CommonConfigurationKeys.HADOOP_SECURITY_TOKEN_SERVICE_USE_IP_DEFAULT);
     setTokenServiceUseIp(useIp);
+    if (HttpConfig.isSecure()) {
+      sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
+      try {
+        sslFactory.init();
+      } catch (Exception ex) {
+        throw new RuntimeException(ex);
+      }
+    }
   }
   
   /**
@@ -456,7 +469,7 @@ public class SecurityUtil {
 
     AuthenticatedURL.Token token = new AuthenticatedURL.Token();
     try {
-      return new AuthenticatedURL().openConnection(url, token);
+      return new AuthenticatedURL(null, sslFactory).openConnection(url, token);
     } catch (AuthenticationException e) {
       throw new IOException("Exception trying to open authenticated connection to "
               + url, e);

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml?rev=1371525&r1=1371524&r2=1371525&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml Thu Aug  9 22:52:56 2012
@@ -1073,4 +1073,14 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.ssl.enabled</name>
+  <value>false</value>
+  <description>
+    Whether to use SSL for the HTTP endpoints. If set to true, the
+    NameNode, DataNode, ResourceManager, NodeManager, HistoryServer and
+    MapReduceAppMaster web UIs will be served over HTTPS instead HTTP.
+  </description>
+</property>
+
 </configuration>

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java?rev=1371525&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java Thu Aug  9 22:52:56 2012
@@ -0,0 +1,114 @@
+/**
+ * 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.http;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import javax.net.ssl.HttpsURLConnection;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.InputStream;
+import java.io.Writer;
+import java.net.URL;
+
+/**
+ * This testcase issues SSL certificates configures the HttpServer to serve
+ * HTTPS using the created certficates and calls an echo servlet using the
+ * corresponding HTTPS URL.
+ */
+public class TestSSLHttpServer extends HttpServerFunctionalTest {
+  private static final String BASEDIR =
+      System.getProperty("test.build.dir", "target/test-dir") + "/" +
+      TestSSLHttpServer.class.getSimpleName();
+
+  static final Log LOG = LogFactory.getLog(TestSSLHttpServer.class);
+  private static HttpServer server;
+  private static URL baseUrl;
+
+  @Before
+  public void setup() throws Exception {
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+    String classpathDir =
+        KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
+    Configuration conf = new Configuration();
+    String keystoresDir = new File(BASEDIR).getAbsolutePath();
+    String sslConfsDir =
+        KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfsDir, conf, false);
+    conf.setBoolean(CommonConfigurationKeysPublic.HADOOP_SSL_ENABLED_KEY, true);
+
+    //we do this trick because the MR AppMaster is started in another VM and
+    //the HttpServer configuration is not loaded from the job.xml but from the
+    //site.xml files in the classpath
+    Writer writer = new FileWriter(classpathDir + "/core-site.xml");
+    conf.writeXml(writer);
+    writer.close();
+
+    conf.setInt(HttpServer.HTTP_MAX_THREADS, 10);
+    server = createServer("test", conf);
+    server.addServlet("echo", "/echo", TestHttpServer.EchoServlet.class);
+    server.start();
+    baseUrl = new URL("https://localhost:" + server.getPort() + "/");
+    LOG.info("HTTP server started: "+ baseUrl);
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    server.stop();
+    String classpathDir =
+        KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
+    new File(classpathDir + "/core-site.xml").delete();
+  }
+  
+
+  @Test
+  public void testEcho() throws Exception {
+    assertEquals("a:b\nc:d\n", 
+        readOut(new URL(baseUrl, "/echo?a=b&c=d")));
+    assertEquals("a:b\nc&lt;:d\ne:&gt;\n", 
+        readOut(new URL(baseUrl, "/echo?a=b&c<=d&e=>")));
+  }
+
+  private static String readOut(URL url) throws Exception {
+    StringBuilder out = new StringBuilder();
+    HttpsURLConnection conn = (HttpsURLConnection) url.openConnection();
+    SSLFactory sslf = new SSLFactory(SSLFactory.Mode.CLIENT, new Configuration());
+    sslf.init();
+    conn.setSSLSocketFactory(sslf.createSSLSocketFactory());
+    InputStream in = conn.getInputStream();
+    byte[] buffer = new byte[64 * 1024];
+    int len = in.read(buffer);
+    while (len > 0) {
+      out.append(new String(buffer, 0, len));
+      len = in.read(buffer);
+    }
+    return out.toString();
+  }
+
+}