فهرست منبع

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 سال پیش
والد
کامیت
d83061b71b
12فایلهای تغییر یافته به همراه224 افزوده شده و 104 حذف شده
  1. 2 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 111 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java
  3. 11 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
  4. 5 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  5. 4 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java
  6. 10 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java
  7. 9 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  8. 56 58
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  9. 9 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  10. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
  11. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobEndNotifier.java
  12. 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

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

@@ -8,6 +8,8 @@ Release 2.3.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-9784. Add a builder for HttpServer. (Junping Du via llu)
+
     HADOOP 9871. Fix intermittent findbugs warnings in DefaultMetricsSystem.
     (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.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.ConfServlet;
@@ -118,18 +119,117 @@ public class HttpServer implements FilterContainer {
   protected final Map<Context, Boolean> defaultContexts =
       new HashMap<Context, Boolean>();
   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_NOT_LIVE = " - not live";
 
   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); */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port, boolean findPort
       ) throws IOException {
     this(name, bindAddress, port, findPort, new Configuration());
   }
-
+  
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, Connector connector) throws IOException {
     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. 
    *        These will be added to any filters.
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
     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.
    * @param conf Configuration 
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf) throws IOException {
     this(name, bindAddress, port, findPort, conf, null, null, null);
   }
 
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl) 
       throws IOException {
@@ -185,6 +288,7 @@ public class HttpServer implements FilterContainer {
    * @param conf Configuration 
    * @param adminsAcl {@link AccessControlList} of the admins
    */
+  @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl, 
       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.
    */
-  protected void defineFilter(Context ctx, String name,
+  public void defineFilter(Context ctx, String name,
       String classname, Map<String,String> parameters, String[] urls) {
 
     FilterHolder holder = new FilterHolder();
@@ -568,6 +672,10 @@ public class HttpServer implements FilterContainer {
   public Object getAttribute(String name) {
     return webAppContext.getAttribute(name);
   }
+  
+  public WebAppContext getWebAppContext(){
+    return this.webAppContext;
+  }
 
   /**
    * 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)
       throws IOException {
     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
    */
   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
@@ -137,13 +139,16 @@ public class HttpServerFunctionalTest extends Assert {
    */
   public static HttpServer createServer(String webapp, Configuration conf)
       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)
       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
    * @param webapp the webapp to work with
@@ -154,7 +159,8 @@ public class HttpServerFunctionalTest extends Assert {
    */
   public static HttpServer createServer(String webapp, Configuration conf,
       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")
   public static class LongHeaderServlet extends HttpServlet {
-    @SuppressWarnings("unchecked")
     @Override
     public void doGet(HttpServletRequest request,
                       HttpServletResponse response
@@ -362,7 +361,8 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
     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.start();
     int port = myServer.getPort();
@@ -403,8 +403,9 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
     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.start();
     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");
       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();
       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.net.NetUtils;
 import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 
 /**
  * Encapsulates the HTTP server started by the Journal Service.
@@ -69,16 +69,15 @@ public class JournalNodeHttpServer {
         bindAddr.getHostName()));
 
     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(JspHelper.CURRENT_CONF, conf);
     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();
     int tmpInfoPort = infoSocAddr.getPort();
     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);
     if (conf.getBoolean(DFS_HTTPS_ENABLE_KEY, false)) {
       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;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.HashMap;
@@ -70,65 +71,30 @@ public class NameNodeHttpServer {
   public void start() throws IOException {
     final String infoHost = bindAddress.getHostName();
     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);
     if (certSSL) {
@@ -153,6 +119,38 @@ public class NameNodeHttpServer {
     httpServer.start();
     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 {

+ 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.
     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("name.system.image", checkpointImage);
     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
   public void testReadURL() throws Exception {
     // 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();
     try {
       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 {
     new File(System.getProperty("build.webapps", "build/webapps") + "/test"
         ).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("jobend", "/jobend", JobEndServlet.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
   protected void serviceStart() throws Exception {
     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, 
           ProxyUriUtils.PROXY_PATH_SPEC, WebAppProxyServlet.class);
       proxyServer.setAttribute(FETCHER_ATTRIBUTE, fetcher);