Browse Source

HADOOP-9784. Add a builder for HttpServer. (Junping Du via llu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1516130 13f79535-47bb-0310-9956-ffa450edef68
Luke Lu 11 years ago
parent
commit
d83061b71b

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

@@ -8,6 +8,8 @@ Release 2.3.0 - UNRELEASED
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
+    HADOOP-9784. Add a builder for HttpServer. (Junping Du via llu)
+
     HADOOP 9871. Fix intermittent findbugs warnings in DefaultMetricsSystem.
     HADOOP 9871. Fix intermittent findbugs warnings in DefaultMetricsSystem.
     (Junping Du via llu)
     (Junping Du via llu)
 
 

+ 111 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java

@@ -46,6 +46,7 @@ import javax.servlet.http.HttpServletResponse;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.ConfServlet;
 import org.apache.hadoop.conf.ConfServlet;
@@ -118,18 +119,117 @@ public class HttpServer implements FilterContainer {
   protected final Map<Context, Boolean> defaultContexts =
   protected final Map<Context, Boolean> defaultContexts =
       new HashMap<Context, Boolean>();
       new HashMap<Context, Boolean>();
   protected final List<String> filterNames = new ArrayList<String>();
   protected final List<String> filterNames = new ArrayList<String>();
-  private static final int MAX_RETRIES = 10;
   static final String STATE_DESCRIPTION_ALIVE = " - alive";
   static final String STATE_DESCRIPTION_ALIVE = " - alive";
   static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
   static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
 
 
   private final boolean listenerStartedExternally;
   private final boolean listenerStartedExternally;
   
   
+  /**
+   * Class to construct instances of HTTP server with specific options.
+   */
+  public static class Builder {
+    String name;
+    String bindAddress;
+    Integer port;
+    Boolean findPort;
+    Configuration conf;
+    Connector connector;
+    String[] pathSpecs;
+    AccessControlList adminsAcl;
+    boolean securityEnabled = false;
+    String usernameConfKey = null;
+    String keytabConfKey = null;
+    
+    public Builder setName(String name){
+      this.name = name;
+      return this;
+    }
+    
+    public Builder setBindAddress(String bindAddress){
+      this.bindAddress = bindAddress;
+      return this;
+    }
+    
+    public Builder setPort(int port) {
+      this.port = port;
+      return this;
+    }
+    
+    public Builder setFindPort(boolean findPort) {
+      this.findPort = findPort;
+      return this;
+    }
+    
+    public Builder setConf(Configuration conf) {
+      this.conf = conf;
+      return this;
+    }
+    
+    public Builder setConnector(Connector connector) {
+      this.connector = connector;
+      return this;
+    }
+    
+    public Builder setPathSpec(String[] pathSpec) {
+      this.pathSpecs = pathSpec;
+      return this;
+    }
+    
+    public Builder setACL(AccessControlList acl) {
+      this.adminsAcl = acl;
+      return this;
+    }
+    
+    public Builder setSecurityEnabled(boolean securityEnabled) {
+      this.securityEnabled = securityEnabled;
+      return this;
+    }
+    
+    public Builder setUsernameConfKey(String usernameConfKey) {
+      this.usernameConfKey = usernameConfKey;
+      return this;
+    }
+    
+    public Builder setKeytabConfKey(String keytabConfKey) {
+      this.keytabConfKey = keytabConfKey;
+      return this;
+    }
+    
+    public HttpServer build() throws IOException {
+      if (this.name == null) {
+        throw new HadoopIllegalArgumentException("name is not set");
+      }
+      if (this.bindAddress == null) {
+        throw new HadoopIllegalArgumentException("bindAddress is not set");
+      }
+      if (this.port == null) {
+        throw new HadoopIllegalArgumentException("port is not set");
+      }
+      if (this.findPort == null) {
+        throw new HadoopIllegalArgumentException("findPort is not set");
+      }
+      
+      if (this.conf == null) {
+        conf = new Configuration();
+      }
+      
+      HttpServer server = new HttpServer(this.name, this.bindAddress, this.port,
+      this.findPort, this.conf, this.adminsAcl, this.connector, this.pathSpecs);
+      if (this.securityEnabled) {
+        server.initSpnego(this.conf, this.usernameConfKey, this.keytabConfKey);
+      }
+      return server;
+    }
+  }
+  
   /** Same as this(name, bindAddress, port, findPort, null); */
   /** Same as this(name, bindAddress, port, findPort, null); */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port, boolean findPort
   public HttpServer(String name, String bindAddress, int port, boolean findPort
       ) throws IOException {
       ) throws IOException {
     this(name, bindAddress, port, findPort, new Configuration());
     this(name, bindAddress, port, findPort, new Configuration());
   }
   }
-
+  
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, Connector connector) throws IOException {
       boolean findPort, Configuration conf, Connector connector) throws IOException {
     this(name, bindAddress, port, findPort, conf, null, connector, null);
     this(name, bindAddress, port, findPort, conf, null, connector, null);
@@ -149,6 +249,7 @@ public class HttpServer implements FilterContainer {
    * @param pathSpecs Path specifications that this httpserver will be serving. 
    * @param pathSpecs Path specifications that this httpserver will be serving. 
    *        These will be added to any filters.
    *        These will be added to any filters.
    */
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
       boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
     this(name, bindAddress, port, findPort, conf, null, null, pathSpecs);
     this(name, bindAddress, port, findPort, conf, null, null, pathSpecs);
@@ -163,11 +264,13 @@ public class HttpServer implements FilterContainer {
    *        increment by 1 until it finds a free port.
    *        increment by 1 until it finds a free port.
    * @param conf Configuration 
    * @param conf Configuration 
    */
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf) throws IOException {
       boolean findPort, Configuration conf) throws IOException {
     this(name, bindAddress, port, findPort, conf, null, null, null);
     this(name, bindAddress, port, findPort, conf, null, null, null);
   }
   }
 
 
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl) 
       boolean findPort, Configuration conf, AccessControlList adminsAcl) 
       throws IOException {
       throws IOException {
@@ -185,6 +288,7 @@ public class HttpServer implements FilterContainer {
    * @param conf Configuration 
    * @param conf Configuration 
    * @param adminsAcl {@link AccessControlList} of the admins
    * @param adminsAcl {@link AccessControlList} of the admins
    */
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl, 
       boolean findPort, Configuration conf, AccessControlList adminsAcl, 
       Connector connector) throws IOException {
       Connector connector) throws IOException {
@@ -528,7 +632,7 @@ public class HttpServer implements FilterContainer {
   /**
   /**
    * Define a filter for a context and set up default url mappings.
    * Define a filter for a context and set up default url mappings.
    */
    */
-  protected void defineFilter(Context ctx, String name,
+  public void defineFilter(Context ctx, String name,
       String classname, Map<String,String> parameters, String[] urls) {
       String classname, Map<String,String> parameters, String[] urls) {
 
 
     FilterHolder holder = new FilterHolder();
     FilterHolder holder = new FilterHolder();
@@ -568,6 +672,10 @@ public class HttpServer implements FilterContainer {
   public Object getAttribute(String name) {
   public Object getAttribute(String name) {
     return webAppContext.getAttribute(name);
     return webAppContext.getAttribute(name);
   }
   }
+  
+  public WebAppContext getWebAppContext(){
+    return this.webAppContext;
+  }
 
 
   /**
   /**
    * Get the pathname to the webapps files.
    * Get the pathname to the webapps files.

+ 11 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java

@@ -116,7 +116,8 @@ public class HttpServerFunctionalTest extends Assert {
   public static HttpServer createServer(String host, int port)
   public static HttpServer createServer(String host, int port)
       throws IOException {
       throws IOException {
     prepareTestWebapp();
     prepareTestWebapp();
-    return new HttpServer(TEST, host, port, true);
+    return new HttpServer.Builder().setName(TEST).setBindAddress(host)
+        .setPort(port).setFindPort(true).build();
   }
   }
 
 
   /**
   /**
@@ -126,7 +127,8 @@ public class HttpServerFunctionalTest extends Assert {
    * @throws IOException if it could not be created
    * @throws IOException if it could not be created
    */
    */
   public static HttpServer createServer(String webapp) throws IOException {
   public static HttpServer createServer(String webapp) throws IOException {
-    return new HttpServer(webapp, "0.0.0.0", 0, true);
+    return new HttpServer.Builder().setName(webapp).setBindAddress("0.0.0.0")
+        .setPort(0).setFindPort(true).build();
   }
   }
   /**
   /**
    * Create an HttpServer instance for the given webapp
    * Create an HttpServer instance for the given webapp
@@ -137,13 +139,16 @@ public class HttpServerFunctionalTest extends Assert {
    */
    */
   public static HttpServer createServer(String webapp, Configuration conf)
   public static HttpServer createServer(String webapp, Configuration conf)
       throws IOException {
       throws IOException {
-    return new HttpServer(webapp, "0.0.0.0", 0, true, conf);
+    return new HttpServer.Builder().setName(webapp).setBindAddress("0.0.0.0")
+        .setPort(0).setFindPort(true).setConf(conf).build();
   }
   }
 
 
   public static HttpServer createServer(String webapp, Configuration conf, AccessControlList adminsAcl)
   public static HttpServer createServer(String webapp, Configuration conf, AccessControlList adminsAcl)
       throws IOException {
       throws IOException {
-    return new HttpServer(webapp, "0.0.0.0", 0, true, conf, adminsAcl);
+    return new HttpServer.Builder().setName(webapp).setBindAddress("0.0.0.0")
+        .setPort(0).setFindPort(true).setConf(conf).setACL(adminsAcl).build();
   }
   }
+  
   /**
   /**
    * Create an HttpServer instance for the given webapp
    * Create an HttpServer instance for the given webapp
    * @param webapp the webapp to work with
    * @param webapp the webapp to work with
@@ -154,7 +159,8 @@ public class HttpServerFunctionalTest extends Assert {
    */
    */
   public static HttpServer createServer(String webapp, Configuration conf,
   public static HttpServer createServer(String webapp, Configuration conf,
       String[] pathSpecs) throws IOException {
       String[] pathSpecs) throws IOException {
-    return new HttpServer(webapp, "0.0.0.0", 0, true, conf, pathSpecs);
+    return new HttpServer.Builder().setName(webapp).setBindAddress("0.0.0.0")
+        .setPort(0).setFindPort(true).setConf(conf).setPathSpec(pathSpecs).build();
   }
   }
 
 
   /**
   /**

+ 5 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -121,7 +121,6 @@ public class TestHttpServer extends HttpServerFunctionalTest {
 
 
   @SuppressWarnings("serial")
   @SuppressWarnings("serial")
   public static class LongHeaderServlet extends HttpServlet {
   public static class LongHeaderServlet extends HttpServlet {
-    @SuppressWarnings("unchecked")
     @Override
     @Override
     public void doGet(HttpServletRequest request,
     public void doGet(HttpServletRequest request,
                       HttpServletResponse response
                       HttpServletResponse response
@@ -362,7 +361,8 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
     MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
     MyGroupsProvider.mapping.put("userB", Arrays.asList("groupB"));
     MyGroupsProvider.mapping.put("userB", Arrays.asList("groupB"));
 
 
-    HttpServer myServer = new HttpServer("test", "0.0.0.0", 0, true, conf);
+    HttpServer myServer = new HttpServer.Builder().setName("test")
+        .setBindAddress("0.0.0.0").setPort(0).setFindPort(true).build();
     myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.start();
     myServer.start();
     int port = myServer.getPort();
     int port = myServer.getPort();
@@ -403,8 +403,9 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
     MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
     MyGroupsProvider.mapping.put("userE", Arrays.asList("groupE"));
     MyGroupsProvider.mapping.put("userE", Arrays.asList("groupE"));
 
 
-    HttpServer myServer = new HttpServer("test", "0.0.0.0", 0, true, conf,
-        new AccessControlList("userA,userB groupC,groupD"));
+    HttpServer myServer = new HttpServer.Builder().setName("test")
+        .setBindAddress("0.0.0.0").setPort(0).setFindPort(true).setConf(conf)
+        .setACL(new AccessControlList("userA,userB groupC,groupD")).build();
     myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.start();
     myServer.start();
     int port = myServer.getPort();
     int port = myServer.getPort();

+ 4 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java

@@ -42,7 +42,10 @@ public class TestLogLevel extends TestCase {
       log.error("log.error1");
       log.error("log.error1");
       assertTrue(!Level.ERROR.equals(log.getEffectiveLevel()));
       assertTrue(!Level.ERROR.equals(log.getEffectiveLevel()));
 
 
-      HttpServer server = new HttpServer("..", "localhost", 22222, true);
+      HttpServer server = new HttpServer.Builder().setName("..")
+          .setBindAddress("localhost").setPort(22222).setFindPort(true)
+          .build();
+      
       server.start();
       server.start();
       int port = server.getPort();
       int port = server.getPort();
 
 

+ 10 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java

@@ -36,8 +36,8 @@ import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.http.HttpServer;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 
 
 /**
 /**
  * Encapsulates the HTTP server started by the Journal Service.
  * Encapsulates the HTTP server started by the Journal Service.
@@ -69,16 +69,15 @@ public class JournalNodeHttpServer {
         bindAddr.getHostName()));
         bindAddr.getHostName()));
 
 
     int tmpInfoPort = bindAddr.getPort();
     int tmpInfoPort = bindAddr.getPort();
-    httpServer = new HttpServer("journal", bindAddr.getHostName(),
-        tmpInfoPort, tmpInfoPort == 0, conf, new AccessControlList(conf
-            .get(DFS_ADMIN, " "))) {
-      {
-        if (UserGroupInformation.isSecurityEnabled()) {
-          initSpnego(conf, DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY,
-              DFSUtil.getSpnegoKeytabKey(conf, DFS_JOURNALNODE_KEYTAB_FILE_KEY));
-        }
-      }
-    };
+    httpServer = new HttpServer.Builder().setName("journal")
+        .setBindAddress(bindAddr.getHostName()).setPort(tmpInfoPort)
+        .setFindPort(tmpInfoPort == 0).setConf(conf).setACL(
+            new AccessControlList(conf.get(DFS_ADMIN, " ")))
+        .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
+        .setUsernameConfKey(
+            DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY)
+        .setKeytabConfKey(DFSUtil.getSpnegoKeytabKey(conf,
+            DFS_JOURNALNODE_KEYTAB_FILE_KEY)).build();
     httpServer.setAttribute(JN_ATTRIBUTE_KEY, localJournalNode);
     httpServer.setAttribute(JN_ATTRIBUTE_KEY, localJournalNode);
     httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     httpServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     httpServer.addInternalServlet("getJournal", "/getJournal",
     httpServer.addInternalServlet("getJournal", "/getJournal",

+ 9 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -390,11 +390,15 @@ public class DataNode extends Configured
     String infoHost = infoSocAddr.getHostName();
     String infoHost = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
     int tmpInfoPort = infoSocAddr.getPort();
     this.infoServer = (secureResources == null) 
     this.infoServer = (secureResources == null) 
-       ? new HttpServer("datanode", infoHost, tmpInfoPort, tmpInfoPort == 0, 
-           conf, new AccessControlList(conf.get(DFS_ADMIN, " ")))
-       : new HttpServer("datanode", infoHost, tmpInfoPort, tmpInfoPort == 0,
-           conf, new AccessControlList(conf.get(DFS_ADMIN, " ")),
-           secureResources.getListener());
+        ? new HttpServer.Builder().setName("datanode")
+            .setBindAddress(infoHost).setPort(tmpInfoPort)
+            .setFindPort(tmpInfoPort == 0).setConf(conf)
+            .setACL(new AccessControlList(conf.get(DFS_ADMIN, " "))).build()
+        : new HttpServer.Builder().setName("datanode")
+            .setBindAddress(infoHost).setPort(tmpInfoPort)
+            .setFindPort(tmpInfoPort == 0).setConf(conf)
+            .setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
+            .setConnector(secureResources.getListener()).build();
     LOG.info("Opened info server at " + infoHost + ":" + tmpInfoPort);
     LOG.info("Opened info server at " + infoHost + ":" + tmpInfoPort);
     if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
     if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
       boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,
       boolean needClientAuth = conf.getBoolean(DFS_CLIENT_HTTPS_NEED_AUTH_KEY,

+ 56 - 58
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
+
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -70,65 +71,30 @@ public class NameNodeHttpServer {
   public void start() throws IOException {
   public void start() throws IOException {
     final String infoHost = bindAddress.getHostName();
     final String infoHost = bindAddress.getHostName();
     int infoPort = bindAddress.getPort();
     int infoPort = bindAddress.getPort();
-
-    httpServer = new HttpServer("hdfs", infoHost, infoPort,
-                                infoPort == 0, conf,
-                                new AccessControlList(conf.get(DFS_ADMIN, " "))) {
-      {
-        // Add SPNEGO support to NameNode
-        if (UserGroupInformation.isSecurityEnabled()) {
-          initSpnego(conf,
-              DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
-              DFSUtil.getSpnegoKeytabKey(conf,
-                  DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
-        }
-        if (WebHdfsFileSystem.isEnabled(conf, LOG)) {
-          //add SPNEGO authentication filter for webhdfs
-          final String name = "SPNEGO";
-          final String classname = AuthFilter.class.getName();
-          final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
-          Map<String, String> params = getAuthFilterParams(conf);
-          defineFilter(webAppContext, name, classname, params,
-                       new String[]{pathSpec});
-          LOG.info("Added filter '" + name + "' (class=" + classname + ")");
-
-          // add webhdfs packages
-          addJerseyResourcePackage(
-            NamenodeWebHdfsMethods.class.getPackage().getName()
-            + ";" + Param.class.getPackage().getName(), pathSpec);
-        }
-      }
-
-      private Map<String, String> getAuthFilterParams(Configuration conf)
-        throws IOException {
-        Map<String, String> params = new HashMap<String, String>();
-        String principalInConf = conf
-          .get(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY);
-        if (principalInConf != null && !principalInConf.isEmpty()) {
-          params
-            .put(
-              DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
-              SecurityUtil.getServerPrincipal(principalInConf,
-                                              bindAddress.getHostName()));
-        } else if (UserGroupInformation.isSecurityEnabled()) {
-          LOG.error("WebHDFS and security are enabled, but configuration property '" +
-                    DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY +
-                    "' is not set.");
-        }
-        String httpKeytab = conf.get(DFSUtil.getSpnegoKeytabKey(conf,
-            DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
-        if (httpKeytab != null && !httpKeytab.isEmpty()) {
-          params.put(
-            DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
-            httpKeytab);
-        } else if (UserGroupInformation.isSecurityEnabled()) {
-          LOG.error("WebHDFS and security are enabled, but configuration property '" +
-                    DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY +
-                    "' is not set.");
-        }
-        return params;
+    httpServer = new HttpServer.Builder().setName("hdfs")
+        .setBindAddress(infoHost).setPort(infoPort)
+        .setFindPort(infoPort == 0).setConf(conf).setACL(
+            new AccessControlList(conf.get(DFS_ADMIN, " ")))
+        .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
+        .setUsernameConfKey(
+            DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY)
+        .setKeytabConfKey(DFSUtil.getSpnegoKeytabKey(conf,
+            DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY)).build();
+    if (WebHdfsFileSystem.isEnabled(conf, HttpServer.LOG)) {
+      //add SPNEGO authentication filter for webhdfs
+      final String name = "SPNEGO";
+      final String classname = AuthFilter.class.getName();
+      final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
+      Map<String, String> params = getAuthFilterParams(conf);
+      httpServer.defineFilter(httpServer.getWebAppContext(), name, classname, params,
+          new String[]{pathSpec});
+      HttpServer.LOG.info("Added filter '" + name + "' (class=" + classname + ")");
+
+      // add webhdfs packages
+      httpServer.addJerseyResourcePackage(
+          NamenodeWebHdfsMethods.class.getPackage().getName()
+          + ";" + Param.class.getPackage().getName(), pathSpec);
       }
       }
-    };
 
 
     boolean certSSL = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, false);
     boolean certSSL = conf.getBoolean(DFSConfigKeys.DFS_HTTPS_ENABLE_KEY, false);
     if (certSSL) {
     if (certSSL) {
@@ -153,6 +119,38 @@ public class NameNodeHttpServer {
     httpServer.start();
     httpServer.start();
     httpAddress = new InetSocketAddress(bindAddress.getAddress(), httpServer.getPort());
     httpAddress = new InetSocketAddress(bindAddress.getAddress(), httpServer.getPort());
   }
   }
+  
+  private Map<String, String> getAuthFilterParams(Configuration conf)
+      throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    String principalInConf = conf
+        .get(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY);
+    if (principalInConf != null && !principalInConf.isEmpty()) {
+      params
+          .put(
+              DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
+              SecurityUtil.getServerPrincipal(principalInConf,
+                                              bindAddress.getHostName()));
+    } else if (UserGroupInformation.isSecurityEnabled()) {
+      HttpServer.LOG.error(
+          "WebHDFS and security are enabled, but configuration property '" +
+          DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY +
+          "' is not set.");
+    }
+    String httpKeytab = conf.get(DFSUtil.getSpnegoKeytabKey(conf,
+        DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
+    if (httpKeytab != null && !httpKeytab.isEmpty()) {
+      params.put(
+          DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
+          httpKeytab);
+    } else if (UserGroupInformation.isSecurityEnabled()) {
+      HttpServer.LOG.error(
+          "WebHDFS and security are enabled, but configuration property '" +
+          DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY +
+          "' is not set.");
+    }
+    return params;
+  }
 
 
 
 
   public void stop() throws Exception {
   public void stop() throws Exception {

+ 9 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -256,19 +256,15 @@ public class SecondaryNameNode implements Runnable {
 
 
     // initialize the webserver for uploading files.
     // initialize the webserver for uploading files.
     int tmpInfoPort = infoSocAddr.getPort();
     int tmpInfoPort = infoSocAddr.getPort();
-    infoServer = new HttpServer("secondary", infoBindAddress, tmpInfoPort,
-                                tmpInfoPort == 0, conf,
-                                new AccessControlList(conf.get(DFS_ADMIN, " "))) {
-      {
-        if (UserGroupInformation.isSecurityEnabled()) {
-          initSpnego(
-              conf,
-              DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
-              DFSUtil.getSpnegoKeytabKey(conf,
-                  DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY));
-        }
-      }
-    };
+    infoServer = new HttpServer.Builder().setName("secondary")
+        .setBindAddress(infoBindAddress).setPort(tmpInfoPort)
+        .setFindPort(tmpInfoPort == 0).setConf(conf).setACL(
+            new AccessControlList(conf.get(DFS_ADMIN, " ")))
+        .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
+        .setUsernameConfKey(
+            DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY)
+        .setKeytabConfKey(DFSUtil.getSpnegoKeytabKey(conf,
+            DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY)).build();
     infoServer.setAttribute("secondary.name.node", this);
     infoServer.setAttribute("secondary.name.node", this);
     infoServer.setAttribute("name.system.image", checkpointImage);
     infoServer.setAttribute("name.system.image", checkpointImage);
     infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java

@@ -41,7 +41,8 @@ public class TestEditLogFileInputStream {
   @Test
   @Test
   public void testReadURL() throws Exception {
   public void testReadURL() throws Exception {
     // Start a simple web server which hosts the log data.
     // Start a simple web server which hosts the log data.
-    HttpServer server = new HttpServer("test", "0.0.0.0", 0, true);
+    HttpServer server = new HttpServer.Builder().setName("test")
+        .setBindAddress("0.0.0.0").setPort(0).setFindPort(true).build();
     server.start();
     server.start();
     try {
     try {
       server.addServlet("fakeLog", "/fakeLog", FakeLogServlet.class);
       server.addServlet("fakeLog", "/fakeLog", FakeLogServlet.class);

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobEndNotifier.java

@@ -102,7 +102,8 @@ public class TestJobEndNotifier extends TestCase {
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     new File(System.getProperty("build.webapps", "build/webapps") + "/test"
     new File(System.getProperty("build.webapps", "build/webapps") + "/test"
         ).mkdirs();
         ).mkdirs();
-    server = new HttpServer("test", "0.0.0.0", 0, true);
+    server = new HttpServer.Builder().setName("test")
+        .setBindAddress("0.0.0.0").setPort(0).setFindPort(true).build();
     server.addServlet("delay", "/delay", DelayServlet.class);
     server.addServlet("delay", "/delay", DelayServlet.class);
     server.addServlet("jobend", "/jobend", JobEndServlet.class);
     server.addServlet("jobend", "/jobend", JobEndServlet.class);
     server.addServlet("fail", "/fail", FailServlet.class);
     server.addServlet("fail", "/fail", FailServlet.class);

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java

@@ -87,8 +87,9 @@ public class WebAppProxy extends AbstractService {
   @Override
   @Override
   protected void serviceStart() throws Exception {
   protected void serviceStart() throws Exception {
     try {
     try {
-      proxyServer = new HttpServer("proxy", bindAddress, port,
-          port == 0, getConfig(), acl);
+      proxyServer = new HttpServer.Builder().setName("proxy")
+          .setBindAddress(bindAddress).setPort(port).setFindPort(port == 0)
+          .setConf(getConfig()).setACL(acl).build();
       proxyServer.addServlet(ProxyUriUtils.PROXY_SERVLET_NAME, 
       proxyServer.addServlet(ProxyUriUtils.PROXY_SERVLET_NAME, 
           ProxyUriUtils.PROXY_PATH_SPEC, WebAppProxyServlet.class);
           ProxyUriUtils.PROXY_PATH_SPEC, WebAppProxyServlet.class);
       proxyServer.setAttribute(FETCHER_ATTRIBUTE, fetcher);
       proxyServer.setAttribute(FETCHER_ATTRIBUTE, fetcher);