Parcourir la source

HADOOP-10255. Merge 1561959 and 1561961 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1561966 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas il y a 11 ans
Parent
commit
afc40e8a66
36 fichiers modifiés avec 218 ajouts et 351 suppressions
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfServlet.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/AdminAuthorizedServlet.java
  4. 62 187
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
  5. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java
  6. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java
  7. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsServlet.java
  8. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java
  9. 17 17
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
  10. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestGlobalFilter.java
  11. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHtmlQuoting.java
  12. 27 38
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
  13. 16 16
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java
  14. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWebapps.java
  15. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java
  16. 3 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
  17. 7 7
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java
  18. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServlet.java
  19. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLogLevel.java
  20. 2 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java
  21. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  22. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeHttpServer.java
  23. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  24. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java
  25. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java
  26. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
  27. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  28. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetImageServlet.java
  29. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
  30. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
  31. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
  32. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobEndNotifier.java
  33. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java
  34. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
  35. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
  36. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java

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

@@ -368,6 +368,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9830. Fix typo at http://hadoop.apache.org/docs/current/
     (Kousuke Saruta via Arpit Agarwal)
 
+    HADOOP-10255. Rename HttpServer to HttpServer2 to retain older 
+    HttpServer in branch-2 for compatibility. (Haohui Mai via suresh)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfServlet.java

@@ -27,7 +27,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 
 /**
  * A servlet to print out the running configuration data.
@@ -47,7 +47,7 @@ public class ConfServlet extends HttpServlet {
    */
   private Configuration getConfFromContext() {
     Configuration conf = (Configuration)getServletContext().getAttribute(
-        HttpServer.CONF_CONTEXT_ATTRIBUTE);
+        HttpServer2.CONF_CONTEXT_ATTRIBUTE);
     assert conf != null;
     return conf;
   }
@@ -56,7 +56,7 @@ public class ConfServlet extends HttpServlet {
   public void doGet(HttpServletRequest request, HttpServletResponse response)
       throws ServletException, IOException {
 
-    if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
+    if (!HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
                                                    request, response)) {
       return;
     }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/AdminAuthorizedServlet.java

@@ -37,7 +37,7 @@ public class AdminAuthorizedServlet extends DefaultServlet {
   protected void doGet(HttpServletRequest request, HttpServletResponse response)
  throws ServletException, IOException {
     // Do the authorization
-    if (HttpServer.hasAdministratorAccess(getServletContext(), request,
+    if (HttpServer2.hasAdministratorAccess(getServletContext(), request,
         response)) {
       // Authorization is done. Just call super.
       super.doGet(request, response);

+ 62 - 187
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java

@@ -24,7 +24,6 @@ import java.io.PrintWriter;
 import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -89,17 +88,19 @@ import com.google.common.collect.Lists;
 import com.sun.jersey.spi.container.servlet.ServletContainer;
 
 /**
- * Create a Jetty embedded server to answer http requests. The primary goal
- * is to serve up status information for the server.
- * There are three contexts:
- *   "/logs/" -> points to the log directory
- *   "/static/" -> points to common static files (src/webapps/static)
- *   "/" -> the jsp server code from (src/webapps/<name>)
+ * Create a Jetty embedded server to answer http requests. The primary goal is
+ * to serve up status information for the server. There are three contexts:
+ * "/logs/" -> points to the log directory "/static/" -> points to common static
+ * files (src/webapps/static) "/" -> the jsp server code from
+ * (src/webapps/<name>)
+ *
+ * This class is a fork of the old HttpServer. HttpServer exists for
+ * compatibility reasons. See HBASE-10336 for more details.
  */
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "HBase"})
+@InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class HttpServer implements FilterContainer {
-  public static final Log LOG = LogFactory.getLog(HttpServer.class);
+public final class HttpServer2 implements FilterContainer {
+  public static final Log LOG = LogFactory.getLog(HttpServer2.class);
 
   static final String FILTER_INITIALIZER_PROPERTY
       = "hadoop.http.filter.initializers";
@@ -166,11 +167,6 @@ public class HttpServer implements FilterContainer {
     // The -keypass option in keytool
     private String keyPassword;
 
-    @Deprecated
-    private String bindAddress;
-    @Deprecated
-    private int port = -1;
-
     private boolean findPort;
 
     private String hostName;
@@ -204,7 +200,7 @@ public class HttpServer implements FilterContainer {
       this.hostName = hostName;
       return this;
     }
-    
+
     public Builder trustStore(String location, String password, String type) {
       this.trustStore = location;
       this.trustStorePassword = password;
@@ -233,78 +229,51 @@ public class HttpServer implements FilterContainer {
       return this;
     }
 
-    /**
-     * Use addEndpoint() instead.
-     */
-    @Deprecated
-    public Builder setBindAddress(String bindAddress){
-      this.bindAddress = bindAddress;
-      return this;
-    }
-
-    /**
-     * Use addEndpoint() instead.
-     */
-    @Deprecated
-    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 {
+
+    public HttpServer2 build() throws IOException {
       if (this.name == null) {
         throw new HadoopIllegalArgumentException("name is not set");
       }
 
-      // Make the behavior compatible with deprecated interfaces
-      if (bindAddress != null && port != -1) {
-        try {
-          endpoints.add(0, new URI("http", "", bindAddress, port, "", "", ""));
-        } catch (URISyntaxException e) {
-          throw new HadoopIllegalArgumentException("Invalid endpoint: "+ e);
-        }
-      }
-
       if (endpoints.size() == 0 && connector == null) {
         throw new HadoopIllegalArgumentException("No endpoints specified");
       }
@@ -313,12 +282,12 @@ public class HttpServer implements FilterContainer {
         hostName = endpoints.size() == 0 ? connector.getHost() : endpoints.get(
             0).getHost();
       }
-      
+
       if (this.conf == null) {
         conf = new Configuration();
       }
-      
-      HttpServer server = new HttpServer(this);
+
+      HttpServer2 server = new HttpServer2(this);
 
       if (this.securityEnabled) {
         server.initSpnego(conf, hostName, usernameConfKey, keytabConfKey);
@@ -332,7 +301,7 @@ public class HttpServer implements FilterContainer {
         Connector listener = null;
         String scheme = ep.getScheme();
         if ("http".equals(scheme)) {
-          listener = HttpServer.createDefaultChannelConnector();
+          listener = HttpServer2.createDefaultChannelConnector();
         } else if ("https".equals(scheme)) {
           SslSocketConnector c = new SslSocketConnector();
           c.setNeedClientAuth(needsClientAuth);
@@ -363,105 +332,8 @@ public class HttpServer implements FilterContainer {
       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);
-  }
-
-  /**
-   * Create a status server on the given port. Allows you to specify the
-   * path specifications that this server will be serving so that they will be
-   * added to the filters properly.  
-   * 
-   * @param name The name of the server
-   * @param bindAddress The address for this server
-   * @param port The port to use on the server
-   * @param findPort whether the server should start at the given port and 
-   *        increment by 1 until it finds a free port.
-   * @param conf Configuration 
-   * @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);
-  }
-  
-  /**
-   * Create a status server on the given port.
-   * The jsp scripts are taken from src/webapps/<name>.
-   * @param name The name of the server
-   * @param port The port to use on the server
-   * @param findPort whether the server should start at the given port and 
-   *        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 {
-    this(name, bindAddress, port, findPort, conf, adminsAcl, null, null);
-  }
-
-  /**
-   * Create a status server on the given port.
-   * The jsp scripts are taken from src/webapps/<name>.
-   * @param name The name of the server
-   * @param bindAddress The address for this server
-   * @param port The port to use on the server
-   * @param findPort whether the server should start at the given port and 
-   *        increment by 1 until it finds a free port.
-   * @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 {
-    this(name, bindAddress, port, findPort, conf, adminsAcl, connector, null);
-  }
-
-  /**
-   * Create a status server on the given port.
-   * The jsp scripts are taken from src/webapps/<name>.
-   * @param name The name of the server
-   * @param bindAddress The address for this server
-   * @param port The port to use on the server
-   * @param findPort whether the server should start at the given port and 
-   *        increment by 1 until it finds a free port.
-   * @param conf Configuration 
-   * @param adminsAcl {@link AccessControlList} of the admins
-   * @param connector A jetty connection listener
-   * @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, AccessControlList adminsAcl, 
-      Connector connector, String[] pathSpecs) throws IOException {
-    this(new Builder().setName(name).hostName(bindAddress)
-        .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
-        .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
-        .setConnector(connector).setPathSpec(pathSpecs));
-  }
-
-  private HttpServer(final Builder b) throws IOException {
+  private HttpServer2(final Builder b) throws IOException {
     final String appDir = getWebAppsPath(b.name);
     this.webServer = new Server();
     this.adminsAcl = b.adminsAcl;
@@ -554,9 +426,9 @@ public class HttpServer implements FilterContainer {
    * listener.
    */
   public Connector createBaseListener(Configuration conf) throws IOException {
-    return HttpServer.createDefaultChannelConnector();
+    return HttpServer2.createDefaultChannelConnector();
   }
-  
+
   @InterfaceAudience.Private
   public static Connector createDefaultChannelConnector() {
     SelectChannelConnector ret = new SelectChannelConnector();
@@ -567,7 +439,7 @@ public class HttpServer implements FilterContainer {
     if(Shell.WINDOWS) {
       // result of setting the SO_REUSEADDR flag is different on Windows
       // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx
-      // without this 2 NN's can start on the same machine and listen on 
+      // without this 2 NN's can start on the same machine and listen on
       // the same port with indeterminate routing of incoming requests to them
       ret.setReuseAddress(false);
     }
@@ -601,7 +473,7 @@ public class HttpServer implements FilterContainer {
    */
   protected void addDefaultApps(ContextHandlerCollection parent,
       final String appDir, Configuration conf) throws IOException {
-    // set up the context for "/logs/" if "hadoop.log.dir" property is defined. 
+    // set up the context for "/logs/" if "hadoop.log.dir" property is defined.
     String logDir = System.getProperty("hadoop.log.dir");
     if (logDir != null) {
       Context logContext = new Context(parent, "/logs");
@@ -628,7 +500,7 @@ public class HttpServer implements FilterContainer {
     setContextAttributes(staticContext, conf);
     defaultContexts.put(staticContext, true);
   }
-  
+
   private void setContextAttributes(Context context, Configuration conf) {
     context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
     context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
@@ -654,10 +526,10 @@ public class HttpServer implements FilterContainer {
   }
 
   /**
-   * Add a context 
+   * Add a context
    * @param pathSpec The path spec for the context
    * @param dir The directory containing the context
-   * @param isFiltered if true, the servlet is added to the filter path mapping 
+   * @param isFiltered if true, the servlet is added to the filter path mapping
    * @throws IOException
    */
   protected void addContext(String pathSpec, String dir, boolean isFiltered) throws IOException {
@@ -680,7 +552,7 @@ public class HttpServer implements FilterContainer {
     webAppContext.setAttribute(name, value);
   }
 
-  /** 
+  /**
    * Add a Jersey resource package.
    * @param packageName The Java package name containing the Jersey resource.
    * @param pathSpec The path spec for the servlet
@@ -709,11 +581,11 @@ public class HttpServer implements FilterContainer {
   }
 
   /**
-   * Add an internal servlet in the server. 
+   * Add an internal servlet in the server.
    * Note: This method is to be used for adding servlets that facilitate
    * internal communication and not for user facing functionality. For
-   * servlets added using this method, filters are not enabled. 
-   * 
+   * servlets added using this method, filters are not enabled.
+   *
    * @param name The name of the servlet (can be passed as null)
    * @param pathSpec The path spec for the servlet
    * @param clazz The servlet class
@@ -725,18 +597,18 @@ public class HttpServer implements FilterContainer {
 
   /**
    * Add an internal servlet in the server, specifying whether or not to
-   * protect with Kerberos authentication. 
+   * protect with Kerberos authentication.
    * Note: This method is to be used for adding servlets that facilitate
    * internal communication and not for user facing functionality. For
    +   * servlets added using this method, filters (except internal Kerberos
-   * filters) are not enabled. 
-   * 
+   * filters) are not enabled.
+   *
    * @param name The name of the servlet (can be passed as null)
    * @param pathSpec The path spec for the servlet
    * @param clazz The servlet class
    * @param requireAuth Require Kerberos authenticate to access servlet
    */
-  public void addInternalServlet(String name, String pathSpec, 
+  public void addInternalServlet(String name, String pathSpec,
       Class<? extends HttpServlet> clazz, boolean requireAuth) {
     ServletHolder holder = new ServletHolder(clazz);
     if (name != null) {
@@ -820,7 +692,7 @@ public class HttpServer implements FilterContainer {
       handler.addFilterMapping(fmap);
     }
   }
-  
+
   /**
    * Get the value in the webapp context.
    * @param name The name of the attribute
@@ -829,7 +701,7 @@ public class HttpServer implements FilterContainer {
   public Object getAttribute(String name) {
     return webAppContext.getAttribute(name);
   }
-  
+
   public WebAppContext getWebAppContext(){
     return this.webAppContext;
   }
@@ -842,7 +714,7 @@ public class HttpServer implements FilterContainer {
    */
   protected String getWebAppsPath(String appName) throws FileNotFoundException {
     URL url = getClass().getClassLoader().getResource("webapps/" + appName);
-    if (url == null) 
+    if (url == null)
       throw new FileNotFoundException("webapps/" + appName
           + " not found in CLASSPATH");
     String urlString = url.toString();
@@ -900,7 +772,7 @@ public class HttpServer implements FilterContainer {
       params.put("kerberos.keytab", httpKeytab);
     }
     params.put(AuthenticationFilter.AUTH_TYPE, "kerberos");
-  
+
     defineFilter(webAppContext, SPNEGO_FILTER,
                  AuthenticationFilter.class.getName(), params, null);
   }
@@ -939,6 +811,9 @@ public class HttpServer implements FilterContainer {
       }
     } catch (IOException e) {
       throw e;
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException(
+          "Interrupted while starting HTTP server").initCause(e);
     } catch (Exception e) {
       throw new IOException("Problem starting http server", e);
     }
@@ -984,7 +859,7 @@ public class HttpServer implements FilterContainer {
       }
     }
   }
-  
+
   /**
    * stop the server
    */
@@ -1102,7 +977,7 @@ public class HttpServer implements FilterContainer {
   /**
    * Does the user sending the HttpServletRequest has the administrator ACLs? If
    * it isn't the case, response will be modified to send an error to the user.
-   * 
+   *
    * @param servletContext
    * @param request
    * @param response used to send the error response if user does not have admin access.
@@ -1127,7 +1002,7 @@ public class HttpServer implements FilterContainer {
                          "authorized to access this page.");
       return false;
     }
-    
+
     if (servletContext.getAttribute(ADMINS_ACL) != null &&
         !userHasAdministratorAccess(servletContext, remoteUser)) {
       response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User "
@@ -1141,7 +1016,7 @@ public class HttpServer implements FilterContainer {
   /**
    * Get the admin ACLs from the given ServletContext and check if the given
    * user is in the ACL.
-   * 
+   *
    * @param servletContext the context containing the admin ACL.
    * @param remoteUser the remote user to check for.
    * @return true if the user is present in the ACL, false if no ACL is set or
@@ -1168,7 +1043,7 @@ public class HttpServer implements FilterContainer {
     @Override
     public void doGet(HttpServletRequest request, HttpServletResponse response)
       throws ServletException, IOException {
-      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
+      if (!HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
                                                      request, response)) {
         return;
       }
@@ -1176,10 +1051,10 @@ public class HttpServer implements FilterContainer {
       PrintWriter out = response.getWriter();
       ReflectionUtils.printThreadInfo(out, "");
       out.close();
-      ReflectionUtils.logThreadInfo(LOG, "jsp requested", 1);      
+      ReflectionUtils.logThreadInfo(LOG, "jsp requested", 1);
     }
   }
-  
+
   /**
    * A Servlet input filter that quotes all HTML active characters in the
    * parameter names and values. The goal is to quote the characters to make
@@ -1194,7 +1069,7 @@ public class HttpServer implements FilterContainer {
         super(rawRequest);
         this.rawRequest = rawRequest;
       }
-      
+
       /**
        * Return the set of parameter names, quoting each name.
        */
@@ -1215,7 +1090,7 @@ public class HttpServer implements FilterContainer {
           }
         };
       }
-      
+
       /**
        * Unquote the name and quote the value.
        */
@@ -1224,7 +1099,7 @@ public class HttpServer implements FilterContainer {
         return HtmlQuoting.quoteHtmlChars(rawRequest.getParameter
                                      (HtmlQuoting.unquoteHtmlChars(name)));
       }
-      
+
       @Override
       public String[] getParameterValues(String name) {
         String unquoteName = HtmlQuoting.unquoteHtmlChars(name);
@@ -1254,7 +1129,7 @@ public class HttpServer implements FilterContainer {
         }
         return result;
       }
-      
+
       /**
        * Quote the url so that users specifying the HOST HTTP header
        * can't inject attacks.
@@ -1264,7 +1139,7 @@ public class HttpServer implements FilterContainer {
         String url = rawRequest.getRequestURL().toString();
         return new StringBuffer(HtmlQuoting.quoteHtmlChars(url));
       }
-      
+
       /**
        * Quote the server name so that users specifying the HOST HTTP header
        * can't inject attacks.
@@ -1285,11 +1160,11 @@ public class HttpServer implements FilterContainer {
     }
 
     @Override
-    public void doFilter(ServletRequest request, 
+    public void doFilter(ServletRequest request,
                          ServletResponse response,
                          FilterChain chain
                          ) throws IOException, ServletException {
-      HttpServletRequestWrapper quoted = 
+      HttpServletRequestWrapper quoted =
         new RequestQuoter((HttpServletRequest) request);
       HttpServletResponse httpResponse = (HttpServletResponse) response;
 

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java

@@ -46,7 +46,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonGenerator;
 
@@ -153,7 +153,7 @@ public class JMXJsonServlet extends HttpServlet {
     String jsonpcb = null;
     PrintWriter writer = null;
     try {
-      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
+      if (!HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
                                                      request, response)) {
         return;
       }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java

@@ -28,7 +28,7 @@ import org.apache.commons.logging.*;
 import org.apache.commons.logging.impl.*;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.util.ServletUtil;
 
 /**
@@ -93,7 +93,7 @@ public class LogLevel {
         ) throws ServletException, IOException {
 
       // Do the authorization
-      if (!HttpServer.hasAdministratorAccess(getServletContext(), request,
+      if (!HttpServer2.hasAdministratorAccess(getServletContext(), request,
           response)) {
         return;
       }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/MetricsServlet.java

@@ -32,7 +32,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.metrics.spi.OutputRecord;
 import org.apache.hadoop.metrics.spi.AbstractMetricsContext.MetricMap;
 import org.apache.hadoop.metrics.spi.AbstractMetricsContext.TagMap;
@@ -106,7 +106,7 @@ public class MetricsServlet extends HttpServlet {
   public void doGet(HttpServletRequest request, HttpServletResponse response)
       throws ServletException, IOException {
 
-    if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
+    if (!HttpServer2.isInstrumentationAccessAllowed(getServletContext(),
                                                    request, response)) {
       return;
     }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/AuthenticationFilterInitializer.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.security;
 
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.FilterContainer;
@@ -94,7 +94,7 @@ public class AuthenticationFilterInitializer extends FilterInitializer {
     }
 
     //Resolve _HOST into bind address
-    String bindAddress = conf.get(HttpServer.BIND_ADDRESS);
+    String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
     String principal = filterConfig.get(KerberosAuthenticationHandler.PRINCIPAL);
     if (principal != null) {
       try {

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

@@ -23,7 +23,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.HttpServer.Builder;
+import org.apache.hadoop.http.HttpServer2.Builder;
 
 import java.io.File;
 import java.io.IOException;
@@ -33,7 +33,7 @@ import java.net.URL;
 import java.net.MalformedURLException;
 
 /**
- * This is a base class for functional tests of the {@link HttpServer}.
+ * This is a base class for functional tests of the {@link HttpServer2}.
  * The methods are static for other classes to import statically.
  */
 public class HttpServerFunctionalTest extends Assert {
@@ -54,7 +54,7 @@ public class HttpServerFunctionalTest extends Assert {
    * @throws IOException if a problem occurs
    * @throws AssertionError if a condition was not met
    */
-  public static HttpServer createTestServer() throws IOException {
+  public static HttpServer2 createTestServer() throws IOException {
     prepareTestWebapp();
     return createServer(TEST);
   }
@@ -68,13 +68,13 @@ public class HttpServerFunctionalTest extends Assert {
    * @throws IOException if a problem occurs
    * @throws AssertionError if a condition was not met
    */
-  public static HttpServer createTestServer(Configuration conf)
+  public static HttpServer2 createTestServer(Configuration conf)
       throws IOException {
     prepareTestWebapp();
     return createServer(TEST, conf);
   }
 
-  public static HttpServer createTestServer(Configuration conf, AccessControlList adminsAcl)
+  public static HttpServer2 createTestServer(Configuration conf, AccessControlList adminsAcl)
       throws IOException {
     prepareTestWebapp();
     return createServer(TEST, conf, adminsAcl);
@@ -89,7 +89,7 @@ public class HttpServerFunctionalTest extends Assert {
    * @throws IOException if a problem occurs
    * @throws AssertionError if a condition was not met
    */
-  public static HttpServer createTestServer(Configuration conf, 
+  public static HttpServer2 createTestServer(Configuration conf,
       String[] pathSpecs) throws IOException {
     prepareTestWebapp();
     return createServer(TEST, conf, pathSpecs);
@@ -120,10 +120,10 @@ public class HttpServerFunctionalTest extends Assert {
    * @return the server
    * @throws IOException if it could not be created
    */
-  public static HttpServer createServer(String host, int port)
+  public static HttpServer2 createServer(String host, int port)
       throws IOException {
     prepareTestWebapp();
-    return new HttpServer.Builder().setName(TEST)
+    return new HttpServer2.Builder().setName(TEST)
         .addEndpoint(URI.create("http://" + host + ":" + port))
         .setFindPort(true).build();
   }
@@ -134,7 +134,7 @@ public class HttpServerFunctionalTest extends Assert {
    * @return the server
    * @throws IOException if it could not be created
    */
-  public static HttpServer createServer(String webapp) throws IOException {
+  public static HttpServer2 createServer(String webapp) throws IOException {
     return localServerBuilder(webapp).setFindPort(true).build();
   }
   /**
@@ -144,18 +144,18 @@ public class HttpServerFunctionalTest extends Assert {
    * @return the server
    * @throws IOException if it could not be created
    */
-  public static HttpServer createServer(String webapp, Configuration conf)
+  public static HttpServer2 createServer(String webapp, Configuration conf)
       throws IOException {
     return localServerBuilder(webapp).setFindPort(true).setConf(conf).build();
   }
 
-  public static HttpServer createServer(String webapp, Configuration conf, AccessControlList adminsAcl)
+  public static HttpServer2 createServer(String webapp, Configuration conf, AccessControlList adminsAcl)
       throws IOException {
     return localServerBuilder(webapp).setFindPort(true).setConf(conf).setACL(adminsAcl).build();
   }
 
   private static Builder localServerBuilder(String webapp) {
-    return new HttpServer.Builder().setName(webapp).addEndpoint(
+    return new HttpServer2.Builder().setName(webapp).addEndpoint(
         URI.create("http://localhost:0"));
   }
   
@@ -167,7 +167,7 @@ public class HttpServerFunctionalTest extends Assert {
    * @return the server
    * @throws IOException if it could not be created
    */
-  public static HttpServer createServer(String webapp, Configuration conf,
+  public static HttpServer2 createServer(String webapp, Configuration conf,
       String[] pathSpecs) throws IOException {
     return localServerBuilder(webapp).setFindPort(true).setConf(conf).setPathSpec(pathSpecs).build();
   }
@@ -180,8 +180,8 @@ public class HttpServerFunctionalTest extends Assert {
    * @throws IOException on any failure
    * @throws AssertionError if a condition was not met
    */
-  public static HttpServer createAndStartTestServer() throws IOException {
-    HttpServer server = createTestServer();
+  public static HttpServer2 createAndStartTestServer() throws IOException {
+    HttpServer2 server = createTestServer();
     server.start();
     return server;
   }
@@ -191,7 +191,7 @@ public class HttpServerFunctionalTest extends Assert {
    * @param server to stop
    * @throws Exception on any failure
    */
-  public static void stop(HttpServer server) throws Exception {
+  public static void stop(HttpServer2 server) throws Exception {
     if (server != null) {
       server.stop();
     }
@@ -203,7 +203,7 @@ public class HttpServerFunctionalTest extends Assert {
    * @return a URL bonded to the base of the server
    * @throws MalformedURLException if the URL cannot be created.
    */
-  public static URL getServerURL(HttpServer server)
+  public static URL getServerURL(HttpServer2 server)
       throws MalformedURLException {
     assertNotNull("No server", server);
     return new URL("http://"

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestGlobalFilter.java

@@ -40,7 +40,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.junit.Test;
 
 public class TestGlobalFilter extends HttpServerFunctionalTest {
-  static final Log LOG = LogFactory.getLog(HttpServer.class);
+  static final Log LOG = LogFactory.getLog(HttpServer2.class);
   static final Set<String> RECORDS = new TreeSet<String>(); 
 
   /** A very simple filter that records accessed uri's */
@@ -106,9 +106,9 @@ public class TestGlobalFilter extends HttpServerFunctionalTest {
     Configuration conf = new Configuration();
     
     //start a http server with CountingFilter
-    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
         RecordingFilter.Initializer.class.getName());
-    HttpServer http = createTestServer(conf);
+    HttpServer2 http = createTestServer(conf);
     http.start();
 
     final String fsckURL = "/fsck";

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHtmlQuoting.java

@@ -68,8 +68,8 @@ public class TestHtmlQuoting {
   @Test
   public void testRequestQuoting() throws Exception {
     HttpServletRequest mockReq = Mockito.mock(HttpServletRequest.class);
-    HttpServer.QuotingInputFilter.RequestQuoter quoter =
-      new HttpServer.QuotingInputFilter.RequestQuoter(mockReq);
+    HttpServer2.QuotingInputFilter.RequestQuoter quoter =
+      new HttpServer2.QuotingInputFilter.RequestQuoter(mockReq);
     
     Mockito.doReturn("a<b").when(mockReq).getParameter("x");
     assertEquals("Test simple param quoting",

+ 27 - 38
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java

@@ -51,7 +51,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.http.HttpServer.QuotingInputFilter.RequestQuoter;
+import org.apache.hadoop.http.HttpServer2.QuotingInputFilter.RequestQuoter;
 import org.apache.hadoop.http.resource.JerseyResource;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Groups;
@@ -70,7 +70,7 @@ import static org.mockito.Mockito.*;
 
 public class TestHttpServer extends HttpServerFunctionalTest {
   static final Log LOG = LogFactory.getLog(TestHttpServer.class);
-  private static HttpServer server;
+  private static HttpServer2 server;
   private static URL baseUrl;
   private static final int MAX_THREADS = 10;
   
@@ -150,7 +150,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
 
   @BeforeClass public static void setup() throws Exception {
     Configuration conf = new Configuration();
-    conf.setInt(HttpServer.HTTP_MAX_THREADS, 10);
+    conf.setInt(HttpServer2.HTTP_MAX_THREADS, 10);
     server = createTestServer(conf);
     server.addServlet("echo", "/echo", EchoServlet.class);
     server.addServlet("echomap", "/echomap", EchoMapServlet.class);
@@ -357,7 +357,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     Configuration conf = new Configuration();
 
     // Authorization is disabled by default
-    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
         DummyFilterInitializer.class.getName());
     conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
         MyGroupsProvider.class.getName());
@@ -366,9 +366,9 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
     MyGroupsProvider.mapping.put("userB", Arrays.asList("groupB"));
 
-    HttpServer myServer = new HttpServer.Builder().setName("test")
+    HttpServer2 myServer = new HttpServer2.Builder().setName("test")
         .addEndpoint(new URI("http://localhost:0")).setFindPort(true).build();
-    myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
+    myServer.setAttribute(HttpServer2.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.start();
     String serverURL = "http://" + NetUtils.getHostPortString(myServer.getConnectorAddress(0)) + "/";
     for (String servlet : new String[] { "conf", "logs", "stacks",
@@ -394,7 +394,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
         true);
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_INSTRUMENTATION_REQUIRES_ADMIN,
         true);
-    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
         DummyFilterInitializer.class.getName());
 
     conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
@@ -407,10 +407,10 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
     MyGroupsProvider.mapping.put("userE", Arrays.asList("groupE"));
 
-    HttpServer myServer = new HttpServer.Builder().setName("test")
+    HttpServer2 myServer = new HttpServer2.Builder().setName("test")
         .addEndpoint(new URI("http://localhost:0")).setFindPort(true).setConf(conf)
         .setACL(new AccessControlList("userA,userB groupC,groupD")).build();
-    myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
+    myServer.setAttribute(HttpServer2.CONF_CONTEXT_ATTRIBUTE, conf);
     myServer.start();
 
     String serverURL = "http://"
@@ -468,39 +468,39 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     Configuration conf = new Configuration();
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false);
     ServletContext context = Mockito.mock(ServletContext.class);
-    Mockito.when(context.getAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE)).thenReturn(conf);
-    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(null);
+    Mockito.when(context.getAttribute(HttpServer2.CONF_CONTEXT_ATTRIBUTE)).thenReturn(conf);
+    Mockito.when(context.getAttribute(HttpServer2.ADMINS_ACL)).thenReturn(null);
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     Mockito.when(request.getRemoteUser()).thenReturn(null);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
 
     //authorization OFF
-    Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
+    Assert.assertTrue(HttpServer2.hasAdministratorAccess(context, request, response));
 
     //authorization ON & user NULL
     response = Mockito.mock(HttpServletResponse.class);
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
-    Assert.assertFalse(HttpServer.hasAdministratorAccess(context, request, response));
+    Assert.assertFalse(HttpServer2.hasAdministratorAccess(context, request, response));
     Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED), Mockito.anyString());
 
     //authorization ON & user NOT NULL & ACLs NULL
     response = Mockito.mock(HttpServletResponse.class);
     Mockito.when(request.getRemoteUser()).thenReturn("foo");
-    Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
+    Assert.assertTrue(HttpServer2.hasAdministratorAccess(context, request, response));
 
     //authorization ON & user NOT NULL & ACLs NOT NULL & user not in ACLs
     response = Mockito.mock(HttpServletResponse.class);
     AccessControlList acls = Mockito.mock(AccessControlList.class);
     Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(false);
-    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
-    Assert.assertFalse(HttpServer.hasAdministratorAccess(context, request, response));
+    Mockito.when(context.getAttribute(HttpServer2.ADMINS_ACL)).thenReturn(acls);
+    Assert.assertFalse(HttpServer2.hasAdministratorAccess(context, request, response));
     Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED), Mockito.anyString());
 
     //authorization ON & user NOT NULL & ACLs NOT NULL & user in in ACLs
     response = Mockito.mock(HttpServletResponse.class);
     Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(true);
-    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
-    Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
+    Mockito.when(context.getAttribute(HttpServer2.ADMINS_ACL)).thenReturn(acls);
+    Assert.assertTrue(HttpServer2.hasAdministratorAccess(context, request, response));
 
   }
 
@@ -508,38 +508,27 @@ public class TestHttpServer extends HttpServerFunctionalTest {
   public void testRequiresAuthorizationAccess() throws Exception {
     Configuration conf = new Configuration();
     ServletContext context = Mockito.mock(ServletContext.class);
-    Mockito.when(context.getAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE)).thenReturn(conf);
+    Mockito.when(context.getAttribute(HttpServer2.CONF_CONTEXT_ATTRIBUTE)).thenReturn(conf);
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
 
     //requires admin access to instrumentation, FALSE by default
-    Assert.assertTrue(HttpServer.isInstrumentationAccessAllowed(context, request, response));
+    Assert.assertTrue(HttpServer2.isInstrumentationAccessAllowed(context, request, response));
 
     //requires admin access to instrumentation, TRUE
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_INSTRUMENTATION_REQUIRES_ADMIN, true);
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
     AccessControlList acls = Mockito.mock(AccessControlList.class);
     Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(false);
-    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
-    Assert.assertFalse(HttpServer.isInstrumentationAccessAllowed(context, request, response));
-  }
-
-  @Test
-  @SuppressWarnings("deprecation")
-  public void testOldConstructor() throws Exception {
-    HttpServer server = new HttpServer("test", "0.0.0.0", 0, false);
-    try {
-      server.start();
-    } finally {
-      server.stop();
-    }
+    Mockito.when(context.getAttribute(HttpServer2.ADMINS_ACL)).thenReturn(acls);
+    Assert.assertFalse(HttpServer2.isInstrumentationAccessAllowed(context, request, response));
   }
 
   @Test public void testBindAddress() throws Exception {
     checkBindAddress("localhost", 0, false).stop();
     // hang onto this one for a bit more testing
-    HttpServer myServer = checkBindAddress("localhost", 0, false);
-    HttpServer myServer2 = null;
+    HttpServer2 myServer = checkBindAddress("localhost", 0, false);
+    HttpServer2 myServer2 = null;
     try { 
       int port = myServer.getConnectorAddress(0).getPort();
       // it's already in use, true = expect a higher port
@@ -558,9 +547,9 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     }
   }
   
-  private HttpServer checkBindAddress(String host, int port, boolean findPort)
+  private HttpServer2 checkBindAddress(String host, int port, boolean findPort)
       throws Exception {
-    HttpServer server = createServer(host, port);
+    HttpServer2 server = createServer(host, port);
     try {
       // not bound, ephemeral should return requested port (0 for ephemeral)
       List<?> listeners = (List<?>) Whitebox.getInternalState(server,
@@ -608,7 +597,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
   public void testHttpServerBuilderWithExternalConnector() throws Exception {
     Connector c = mock(Connector.class);
     doReturn("localhost").when(c).getHost();
-    HttpServer s = new HttpServer.Builder().setName("test").setConnector(c)
+    HttpServer2 s = new HttpServer2.Builder().setName("test").setConnector(c)
         .build();
     s.stop();
   }

+ 16 - 16
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java

@@ -23,18 +23,18 @@ import org.junit.Test;
 public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
 
   /**
-   * Check that a server is alive by probing the {@link HttpServer#isAlive()} method
+   * Check that a server is alive by probing the {@link HttpServer2#isAlive()} method
    * and the text of its toString() description
    * @param server server
    */
-  private void assertAlive(HttpServer server) {
+  private void assertAlive(HttpServer2 server) {
     assertTrue("Server is not alive", server.isAlive());
-    assertToStringContains(server, HttpServer.STATE_DESCRIPTION_ALIVE);
+    assertToStringContains(server, HttpServer2.STATE_DESCRIPTION_ALIVE);
   }
 
-  private void assertNotLive(HttpServer server) {
+  private void assertNotLive(HttpServer2 server) {
     assertTrue("Server should not be live", !server.isAlive());
-    assertToStringContains(server, HttpServer.STATE_DESCRIPTION_NOT_LIVE);
+    assertToStringContains(server, HttpServer2.STATE_DESCRIPTION_NOT_LIVE);
   }
 
   /**
@@ -43,12 +43,12 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
    * @throws Throwable on failure
    */
   @Test public void testCreatedServerIsNotAlive() throws Throwable {
-    HttpServer server = createTestServer();
+    HttpServer2 server = createTestServer();
     assertNotLive(server);
   }
 
   @Test public void testStopUnstartedServer() throws Throwable {
-    HttpServer server = createTestServer();
+    HttpServer2 server = createTestServer();
     stop(server);
   }
 
@@ -59,7 +59,7 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
    */
   @Test
   public void testStartedServerIsAlive() throws Throwable {
-    HttpServer server = null;
+    HttpServer2 server = null;
     server = createTestServer();
     assertNotLive(server);
     server.start();
@@ -78,22 +78,22 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
     requestLogAppender.setName("httprequestlog");
     requestLogAppender.setFilename(System.getProperty("test.build.data", "/tmp/")
         + "jetty-name-yyyy_mm_dd.log");
-    Logger.getLogger(HttpServer.class.getName() + ".test").addAppender(requestLogAppender);
-    HttpServer server = null;
+    Logger.getLogger(HttpServer2.class.getName() + ".test").addAppender(requestLogAppender);
+    HttpServer2 server = null;
     server = createTestServer();
     assertNotLive(server);
     server.start();
     assertAlive(server);
     stop(server);
-    Logger.getLogger(HttpServer.class.getName() + ".test").removeAppender(requestLogAppender);
+    Logger.getLogger(HttpServer2.class.getName() + ".test").removeAppender(requestLogAppender);
   }
 
   /**
-   * Assert that the result of {@link HttpServer#toString()} contains the specific text
+   * Assert that the result of {@link HttpServer2#toString()} contains the specific text
    * @param server server to examine
    * @param text text to search for
    */
-  private void assertToStringContains(HttpServer server, String text) {
+  private void assertToStringContains(HttpServer2 server, String text) {
     String description = server.toString();
     assertTrue("Did not find \"" + text + "\" in \"" + description + "\"",
                description.contains(text));
@@ -105,7 +105,7 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
    * @throws Throwable on failure
    */
   @Test public void testStoppedServerIsNotAlive() throws Throwable {
-    HttpServer server = createAndStartTestServer();
+    HttpServer2 server = createAndStartTestServer();
     assertAlive(server);
     stop(server);
     assertNotLive(server);
@@ -117,7 +117,7 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
    * @throws Throwable on failure
    */
   @Test public void testStoppingTwiceServerIsAllowed() throws Throwable {
-    HttpServer server = createAndStartTestServer();
+    HttpServer2 server = createAndStartTestServer();
     assertAlive(server);
     stop(server);
     assertNotLive(server);
@@ -133,7 +133,7 @@ public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
    */
   @Test
   public void testWepAppContextAfterServerStop() throws Throwable {
-    HttpServer server = null;
+    HttpServer2 server = null;
     String key = "test.attribute.key";
     String value = "test.attribute.value";
     server = createTestServer();

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerWebapps.java

@@ -36,7 +36,7 @@ public class TestHttpServerWebapps extends HttpServerFunctionalTest {
    */
   @Test
   public void testValidServerResource() throws Throwable {
-    HttpServer server = null;
+    HttpServer2 server = null;
     try {
       server = createServer("test");
     } finally {
@@ -51,7 +51,7 @@ public class TestHttpServerWebapps extends HttpServerFunctionalTest {
   @Test
   public void testMissingServerResource() throws Throwable {
     try {
-      HttpServer server = createServer("NoSuchWebapp");
+      HttpServer2 server = createServer("NoSuchWebapp");
       //should not have got here.
       //close the server
       String serverDescription = server.toString();

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestPathFilter.java

@@ -40,7 +40,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.junit.Test;
 
 public class TestPathFilter extends HttpServerFunctionalTest {
-  static final Log LOG = LogFactory.getLog(HttpServer.class);
+  static final Log LOG = LogFactory.getLog(HttpServer2.class);
   static final Set<String> RECORDS = new TreeSet<String>(); 
 
   /** A very simple filter that records accessed uri's */
@@ -107,10 +107,10 @@ public class TestPathFilter extends HttpServerFunctionalTest {
     Configuration conf = new Configuration();
     
     //start a http server with CountingFilter
-    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
         RecordingFilter.Initializer.class.getName());
     String[] pathSpecs = { "/path", "/path/*" };
-    HttpServer http = createTestServer(conf, pathSpecs);
+    HttpServer2 http = createTestServer(conf, pathSpecs);
     http.start();
 
     final String baseURL = "/path";

+ 3 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java

@@ -48,7 +48,7 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
 
   private static final Log LOG = LogFactory.getLog(TestSSLHttpServer.class);
   private static Configuration conf;
-  private static HttpServer server;
+  private static HttpServer2 server;
   private static URL baseUrl;
   private static String keystoresDir;
   private static String sslConfDir;
@@ -57,7 +57,7 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
   @BeforeClass
   public static void setup() throws Exception {
     conf = new Configuration();
-    conf.setInt(HttpServer.HTTP_MAX_THREADS, 10);
+    conf.setInt(HttpServer2.HTTP_MAX_THREADS, 10);
 
     File base = new File(BASEDIR);
     FileUtil.fullyDelete(base);
@@ -73,7 +73,7 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
     clientSslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, sslConf);
     clientSslFactory.init();
 
-    server = new HttpServer.Builder()
+    server = new HttpServer2.Builder()
         .setName("test")
         .addEndpoint(new URI("https://localhost"))
         .setConf(conf)

+ 7 - 7
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestServletFilter.java

@@ -40,7 +40,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 public class TestServletFilter extends HttpServerFunctionalTest {
-  static final Log LOG = LogFactory.getLog(HttpServer.class);
+  static final Log LOG = LogFactory.getLog(HttpServer2.class);
   static volatile String uri = null; 
 
   /** A very simple filter which record the uri filtered. */
@@ -105,9 +105,9 @@ public class TestServletFilter extends HttpServerFunctionalTest {
     Configuration conf = new Configuration();
     
     //start a http server with CountingFilter
-    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
         SimpleFilter.Initializer.class.getName());
-    HttpServer http = createTestServer(conf);
+    HttpServer2 http = createTestServer(conf);
     http.start();
 
     final String fsckURL = "/fsck";
@@ -166,9 +166,9 @@ public class TestServletFilter extends HttpServerFunctionalTest {
   public void testServletFilterWhenInitThrowsException() throws Exception {
     Configuration conf = new Configuration();
     // start a http server with CountingFilter
-    conf.set(HttpServer.FILTER_INITIALIZER_PROPERTY,
+    conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
         ErrorFilter.Initializer.class.getName());
-    HttpServer http = createTestServer(conf);
+    HttpServer2 http = createTestServer(conf);
     try {
       http.start();
       fail("expecting exception");
@@ -186,8 +186,8 @@ public class TestServletFilter extends HttpServerFunctionalTest {
   public void testContextSpecificServletFilterWhenInitThrowsException()
       throws Exception {
     Configuration conf = new Configuration();
-    HttpServer http = createTestServer(conf);
-    HttpServer.defineFilter(http.webAppContext,
+    HttpServer2 http = createTestServer(conf);
+    HttpServer2.defineFilter(http.webAppContext,
         "ErrorFilter", ErrorFilter.class.getName(),
         null, null);
     try {

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/jmx/TestJMXJsonServlet.java

@@ -24,7 +24,7 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.HttpServerFunctionalTest;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -32,7 +32,7 @@ import org.junit.Test;
 
 public class TestJMXJsonServlet extends HttpServerFunctionalTest {
   private   static final Log LOG = LogFactory.getLog(TestJMXJsonServlet.class);
-  private static HttpServer server;
+  private static HttpServer2 server;
   private static URL baseUrl;
 
   @BeforeClass public static void setup() throws Exception {

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

@@ -20,7 +20,7 @@ package org.apache.hadoop.log;
 import java.io.*;
 import java.net.*;
 
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.net.NetUtils;
 
 import junit.framework.TestCase;
@@ -44,7 +44,7 @@ public class TestLogLevel extends TestCase {
       log.error("log.error1");
       assertTrue(!Level.ERROR.equals(log.getEffectiveLevel()));
 
-      HttpServer server = new HttpServer.Builder().setName("..")
+      HttpServer2 server = new HttpServer2.Builder().setName("..")
           .addEndpoint(new URI("http://localhost:0")).setFindPort(true)
           .build();
       

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestAuthenticationFilter.java

@@ -18,7 +18,7 @@ package org.apache.hadoop.security;
 
 
 import junit.framework.TestCase;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.FilterContainer;
@@ -49,7 +49,7 @@ public class TestAuthenticationFilter extends TestCase {
              AuthenticationFilterInitializer.SIGNATURE_SECRET_FILE, 
              secretFile.getAbsolutePath());
 
-    conf.set(HttpServer.BIND_ADDRESS, "barhost");
+    conf.set(HttpServer2.BIND_ADDRESS, "barhost");
     
     FilterContainer container = Mockito.mock(FilterContainer.class);
     Mockito.doAnswer(

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -85,7 +85,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
@@ -1526,7 +1526,7 @@ public class DFSUtil {
     return policy;
   }
 
-  public static HttpServer.Builder loadSslConfToHttpServerBuilder(HttpServer.Builder builder,
+  public static HttpServer2.Builder loadSslConfToHttpServerBuilder(HttpServer2.Builder builder,
       Configuration sslConf) {
     return builder
         .needsClientAuth(
@@ -1562,13 +1562,13 @@ public class DFSUtil {
    * namenode can use to initialize their HTTP / HTTPS server.
    *
    */
-  public static HttpServer.Builder httpServerTemplateForNNAndJN(
+  public static HttpServer2.Builder httpServerTemplateForNNAndJN(
       Configuration conf, final InetSocketAddress httpAddr,
       final InetSocketAddress httpsAddr, String name, String spnegoUserNameKey,
       String spnegoKeytabFileKey) throws IOException {
     HttpConfig.Policy policy = getHttpPolicy(conf);
 
-    HttpServer.Builder builder = new HttpServer.Builder().setName(name)
+    HttpServer2.Builder builder = new HttpServer2.Builder().setName(name)
         .setConf(conf).setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
         .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
         .setUsernameConfKey(spnegoUserNameKey)

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

@@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.net.NetUtils;
 
 /**
@@ -38,7 +38,7 @@ import org.apache.hadoop.net.NetUtils;
 public class JournalNodeHttpServer {
   public static final String JN_ATTRIBUTE_KEY = "localjournal";
 
-  private HttpServer httpServer;
+  private HttpServer2 httpServer;
   private JournalNode localJournalNode;
 
   private final Configuration conf;
@@ -56,7 +56,7 @@ public class JournalNodeHttpServer {
         DFSConfigKeys.DFS_JOURNALNODE_HTTPS_ADDRESS_DEFAULT);
     InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
 
-    HttpServer.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
         httpAddr, httpsAddr, "journal",
         DFSConfigKeys.DFS_JOURNALNODE_INTERNAL_SPNEGO_USER_NAME_KEY,
         DFSConfigKeys.DFS_JOURNALNODE_KEYTAB_FILE_KEY);

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

@@ -121,7 +121,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.ReadaheadPool;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -236,7 +236,7 @@ public class DataNode extends Configured
   private volatile boolean heartbeatsDisabledForTests = false;
   private DataStorage storage = null;
 
-  private HttpServer infoServer = null;
+  private HttpServer2 infoServer = null;
   private int infoPort;
   private int infoSecurePort;
 
@@ -359,7 +359,7 @@ public class DataNode extends Configured
    * Http Policy is decided.
    */
   private void startInfoServer(Configuration conf) throws IOException {
-    HttpServer.Builder builder = new HttpServer.Builder().setName("datanode")
+    HttpServer2.Builder builder = new HttpServer2.Builder().setName("datanode")
         .setConf(conf).setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")));
 
     HttpConfig.Policy policy = DFSUtil.getHttpPolicy(conf);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/SecureDataNodeStarter.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.mortbay.jetty.Connector;
 
@@ -119,7 +119,7 @@ public class SecureDataNodeStarter implements Daemon {
     // certificates if they are communicating through SSL.
     Connector listener = null;
     if (policy.isHttpEnabled()) {
-      listener = HttpServer.createDefaultChannelConnector();
+      listener = HttpServer2.createDefaultChannelConnector();
       InetSocketAddress infoSocAddr = DataNode.getInfoAddr(conf);
       listener.setHost(infoSocAddr.getHostName());
       listener.setPort(infoSocAddr.getPort());

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java

@@ -47,7 +47,7 @@ import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -287,7 +287,7 @@ public class GetImageServlet extends HttpServlet {
       }
     }
     
-    if (HttpServer.userHasAdministratorAccess(context, remoteUser)) {
+    if (HttpServer2.userHasAdministratorAccess(context, remoteUser)) {
       LOG.info("GetImageServlet allowing administrator: " + remoteUser);
       return true;
     }

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

@@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.Param;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -47,7 +47,7 @@ import org.apache.hadoop.security.UserGroupInformation;
  */
 @InterfaceAudience.Private
 public class NameNodeHttpServer {
-  private HttpServer httpServer;
+  private HttpServer2 httpServer;
   private final Configuration conf;
   private final NameNode nn;
   
@@ -68,7 +68,7 @@ public class NameNodeHttpServer {
   }
 
   private void initWebHdfs(Configuration conf) throws IOException {
-    if (WebHdfsFileSystem.isEnabled(conf, HttpServer.LOG)) {
+    if (WebHdfsFileSystem.isEnabled(conf, HttpServer2.LOG)) {
       // set user pattern based on configuration file
       UserParam.setUserPattern(conf.get(DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY, DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT));
       //add SPNEGO authentication filter for webhdfs
@@ -76,9 +76,9 @@ public class NameNodeHttpServer {
       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,
+      HttpServer2.defineFilter(httpServer.getWebAppContext(), name, classname, params,
           new String[]{pathSpec});
-      HttpServer.LOG.info("Added filter '" + name + "' (class=" + classname + ")");
+      HttpServer2.LOG.info("Added filter '" + name + "' (class=" + classname + ")");
 
       // add webhdfs packages
       httpServer.addJerseyResourcePackage(
@@ -102,7 +102,7 @@ public class NameNodeHttpServer {
         DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT);
     InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
 
-    HttpServer.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
         httpAddr, httpsAddr, "hdfs",
         DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
         DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);
@@ -151,7 +151,7 @@ public class NameNodeHttpServer {
               SecurityUtil.getServerPrincipal(principalInConf,
                                               bindAddress.getHostName()));
     } else if (UserGroupInformation.isSecurityEnabled()) {
-      HttpServer.LOG.error(
+      HttpServer2.LOG.error(
           "WebHDFS and security are enabled, but configuration property '" +
           DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY +
           "' is not set.");
@@ -163,7 +163,7 @@ public class NameNodeHttpServer {
           DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
           httpKeytab);
     } else if (UserGroupInformation.isSecurityEnabled()) {
-      HttpServer.LOG.error(
+      HttpServer2.LOG.error(
           "WebHDFS and security are enabled, but configuration property '" +
           DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY +
           "' is not set.");
@@ -213,7 +213,7 @@ public class NameNodeHttpServer {
     httpServer.setAttribute(STARTUP_PROGRESS_ATTRIBUTE_KEY, prog);
   }
 
-  private static void setupServlets(HttpServer httpServer, Configuration conf) {
+  private static void setupServlets(HttpServer2 httpServer, Configuration conf) {
     httpServer.addInternalServlet("startupProgress",
         StartupProgressServlet.PATH_SPEC, StartupProgressServlet.class);
     httpServer.addInternalServlet("getDelegationToken",

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

@@ -65,7 +65,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -113,7 +113,7 @@ public class SecondaryNameNode implements Runnable {
   private Configuration conf;
   private InetSocketAddress nameNodeAddr;
   private volatile boolean shouldRun;
-  private HttpServer infoServer;
+  private HttpServer2 infoServer;
   private URL imageListenURL;
 
   private Collection<URI> checkpointDirs;
@@ -257,7 +257,7 @@ public class SecondaryNameNode implements Runnable {
         DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_DEFAULT);
     InetSocketAddress httpsAddr = NetUtils.createSocketAddr(httpsAddrString);
 
-    HttpServer.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
+    HttpServer2.Builder builder = DFSUtil.httpServerTemplateForNNAndJN(conf,
         httpAddr, httpsAddr, "secondary",
         DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
         DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY);

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

@@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.authorize.AccessControlList;
@@ -66,7 +66,7 @@ public class TestGetImageServlet {
     AccessControlList acls = Mockito.mock(AccessControlList.class);
     Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(false);
     ServletContext context = Mockito.mock(ServletContext.class);
-    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
+    Mockito.when(context.getAttribute(HttpServer2.ADMINS_ACL)).thenReturn(acls);
     
     // Make sure that NN2 is considered a valid fsimage/edits requestor.
     assertTrue(GetImageServlet.isValidRequestor(context,

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

@@ -37,7 +37,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.HttpServerFunctionalTest;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -119,7 +119,7 @@ public class TestTransferFsImage {
    */
   @Test(timeout = 5000)
   public void testImageTransferTimeout() throws Exception {
-    HttpServer testServer = HttpServerFunctionalTest.createServer("hdfs");
+    HttpServer2 testServer = HttpServerFunctionalTest.createServer("hdfs");
     try {
       testServer.addServlet("GetImage", "/getimage", TestGetImageServlet.class);
       testServer.start();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java

@@ -58,7 +58,7 @@ import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.ipc.ProtobufRpcEngine.Server;
 import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -89,7 +89,7 @@ public class SnapshotTestHelper {
     setLevel2OFF(LogFactory.getLog(MetricsSystemImpl.class));
     
     setLevel2OFF(DataBlockScanner.LOG);
-    setLevel2OFF(HttpServer.LOG);
+    setLevel2OFF(HttpServer2.LOG);
     setLevel2OFF(DataNode.LOG);
     setLevel2OFF(BlockPoolSliceStorage.LOG);
     setLevel2OFF(LeaseManager.LOG);

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java

@@ -37,7 +37,7 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobContext;
 import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -199,7 +199,7 @@ public class TestJobEndNotifier extends JobEndNotifier {
 
   @Test
   public void testNotificationOnLastRetryNormalShutdown() throws Exception {
-    HttpServer server = startHttpServer();
+    HttpServer2 server = startHttpServer();
     // Act like it is the second attempt. Default max attempts is 2
     MRApp app = spy(new MRAppWithCustomContainerAllocator(
         2, 2, true, this.getClass().getName(), true, 2, true));
@@ -223,7 +223,7 @@ public class TestJobEndNotifier extends JobEndNotifier {
   @Test
   public void testAbsentNotificationOnNotLastRetryUnregistrationFailure()
       throws Exception {
-    HttpServer server = startHttpServer();
+    HttpServer2 server = startHttpServer();
     MRApp app = spy(new MRAppWithCustomContainerAllocator(2, 2, false,
         this.getClass().getName(), true, 1, false));
     doNothing().when(app).sysexit();
@@ -250,7 +250,7 @@ public class TestJobEndNotifier extends JobEndNotifier {
   @Test
   public void testNotificationOnLastRetryUnregistrationFailure()
       throws Exception {
-    HttpServer server = startHttpServer();
+    HttpServer2 server = startHttpServer();
     MRApp app = spy(new MRAppWithCustomContainerAllocator(2, 2, false,
         this.getClass().getName(), true, 2, false));
     doNothing().when(app).sysexit();
@@ -274,10 +274,10 @@ public class TestJobEndNotifier extends JobEndNotifier {
     server.stop();
   }
 
-  private static HttpServer startHttpServer() throws Exception {
+  private static HttpServer2 startHttpServer() throws Exception {
     new File(System.getProperty(
         "build.webapps", "build/webapps") + "/test").mkdirs();
-    HttpServer server = new HttpServer.Builder().setName("test")
+    HttpServer2 server = new HttpServer2.Builder().setName("test")
         .addEndpoint(URI.create("http://localhost:0"))
         .setFindPort(true).build();
     server.addServlet("jobend", "/jobend", JobEndServlet.class);

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

@@ -34,10 +34,10 @@ import javax.servlet.http.HttpServletResponse;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 
 public class TestJobEndNotifier extends TestCase {
-  HttpServer server;
+  HttpServer2 server;
   URL baseUrl;
 
   @SuppressWarnings("serial")
@@ -102,7 +102,7 @@ public class TestJobEndNotifier extends TestCase {
   public void setUp() throws Exception {
     new File(System.getProperty("build.webapps", "build/webapps") + "/test"
         ).mkdirs();
-    server = new HttpServer.Builder().setName("test")
+    server = new HttpServer2.Builder().setName("test")
         .addEndpoint(URI.create("http://localhost:0"))
         .setFindPort(true).build();
     server.addServlet("delay", "/delay", DelayServlet.class);

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java

@@ -28,7 +28,7 @@ import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,7 +59,7 @@ public abstract class WebApp extends ServletModule {
   private volatile String redirectPath;
   private volatile String wsName;
   private volatile Configuration conf;
-  private volatile HttpServer httpServer;
+  private volatile HttpServer2 httpServer;
   private volatile GuiceFilter guiceFilter;
   private final Router router = new Router();
 
@@ -72,11 +72,11 @@ public abstract class WebApp extends ServletModule {
   static final Splitter pathSplitter =
       Splitter.on('/').trimResults().omitEmptyStrings();
 
-  void setHttpServer(HttpServer server) {
+  void setHttpServer(HttpServer2 server) {
     httpServer = checkNotNull(server, "http server");
   }
 
-  @Provides public HttpServer httpServer() { return httpServer; }
+  @Provides public HttpServer2 httpServer() { return httpServer; }
 
   /**
    * Get the address the http server is bound to

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java

@@ -35,7 +35,7 @@ import javax.servlet.http.HttpServlet;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.security.AdminACLsManager;
 import org.slf4j.Logger;
@@ -216,7 +216,7 @@ public class WebApps {
             System.exit(1);
           }
         }
-        HttpServer.Builder builder = new HttpServer.Builder().setName(name)
+        HttpServer2.Builder builder = new HttpServer2.Builder().setName(name)
             .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
             .setConf(conf).setFindPort(findPort)
             .setACL(new AdminACLsManager(conf).getAdminAcl())
@@ -229,7 +229,7 @@ public class WebApps {
               .setKeytabConfKey(spnegoKeytabKey)
               .setSecurityEnabled(UserGroupInformation.isSecurityEnabled());
         }
-        HttpServer server = builder.build();
+        HttpServer2 server = builder.build();
 
         for(ServletStruct struct: servlets) {
           server.addServlet(struct.name, struct.spec, struct.clazz);
@@ -237,7 +237,7 @@ public class WebApps {
         for(Map.Entry<String, Object> entry : attributes.entrySet()) {
           server.setAttribute(entry.getKey(), entry.getValue());
         }
-        HttpServer.defineFilter(server.getWebAppContext(), "guice",
+        HttpServer2.defineFilter(server.getWebAppContext(), "guice",
           GuiceFilter.class.getName(), null, new String[] { "/*" });
 
         webapp.setConf(conf);

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

@@ -24,7 +24,7 @@ import java.net.URI;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
@@ -41,7 +41,7 @@ public class WebAppProxy extends AbstractService {
   public static final String PROXY_HOST_ATTRIBUTE = "proxyHost";
   private static final Log LOG = LogFactory.getLog(WebAppProxy.class);
   
-  private HttpServer proxyServer = null;
+  private HttpServer2 proxyServer = null;
   private String bindAddress = null;
   private int port = 0;
   private AccessControlList acl = null;
@@ -90,7 +90,7 @@ public class WebAppProxy extends AbstractService {
   @Override
   protected void serviceStart() throws Exception {
     try {
-      proxyServer = new HttpServer.Builder().setName("proxy")
+      proxyServer = new HttpServer2.Builder().setName("proxy")
           .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
           .setFindPort(port == 0)
           .setConf(getConfig()).setACL(acl).build();

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java

@@ -43,7 +43,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.StringUtils;
@@ -274,7 +274,7 @@ public class TestWebAppProxyServlet {
 
   private class WebAppProxyForTest extends WebAppProxy {
     
-    HttpServer proxyServer;
+    HttpServer2 proxyServer;
     AppReportFetcherForTest appReportFetcher;
     
     @Override
@@ -286,7 +286,7 @@ public class TestWebAppProxyServlet {
         AccessControlList acl = new AccessControlList(
             conf.get(YarnConfiguration.YARN_ADMIN_ACL, 
             YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
-        proxyServer = new HttpServer.Builder()
+        proxyServer = new HttpServer2.Builder()
             .setName("proxy")
             .addEndpoint(URI.create("http://" + bindAddress + ":0"))
             .setFindPort(true)