浏览代码

HADOOP-3854. Add support for pluggable servlet filters in the HttpServers.
(Tsz Wo (Nicholas) Sze via omalley)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@688920 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 17 年之前
父节点
当前提交
cd5bfa302a

+ 3 - 0
CHANGES.txt

@@ -94,6 +94,9 @@ Trunk (unreleased changes)
 
 
     HADOOP-1480. Add counters to the C++ Pipes API. (acmurthy via omalley)
     HADOOP-1480. Add counters to the C++ Pipes API. (acmurthy via omalley)
 
 
+    HADOOP-3854. Add support for pluggable servlet filters in the HttpServers.
+    (Tsz Wo (Nicholas) Sze via omalley)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HADOOP-3732. Delay intialization of datanode block verification till
     HADOOP-3732. Delay intialization of datanode block verification till

+ 11 - 0
conf/hadoop-default.xml

@@ -21,6 +21,17 @@
   <description>Should native hadoop libraries, if present, be used.</description>
   <description>Should native hadoop libraries, if present, be used.</description>
 </property>
 </property>
 
 
+<property>
+  <name>hadoop.http.filter.initializers</name>
+  <value></value>
+  <description>A comma separated list of class names.  Each class in the list must extend org.apache.hadoop.http.FilterInitializer.
+  
+  The corresponding Filter will be initialized.  Then, the Filter will be applied to all user facing jsp and servlet web pages.  The ordering of the list defines the ordering of the filters.
+
+  The value can be empty.
+  </description>
+</property>
+
 <!--- logging properties -->
 <!--- logging properties -->
 
 
 <property>
 <property>

+ 27 - 0
src/core/org/apache/hadoop/conf/Configuration.java

@@ -651,6 +651,33 @@ public class Configuration implements Iterable<Map.Entry<String,String>> {
     return Class.forName(name, true, classLoader);
     return Class.forName(name, true, classLoader);
   }
   }
 
 
+  /** 
+   * Get the value of the <code>name</code> property
+   * as an array of <code>Class</code>.
+   * The value of the property specifies a list of comma separated class names.  
+   * If no such property is specified, then <code>defaultValue</code> is 
+   * returned.
+   * 
+   * @param name the property name.
+   * @param defaultValue default value.
+   * @return property value as a <code>Class[]</code>, 
+   *         or <code>defaultValue</code>. 
+   */
+  public Class<?>[] getClasses(String name, Class<?> ... defaultValue) {
+    String[] classnames = getStrings(name);
+    if (classnames == null)
+      return defaultValue;
+    try {
+      Class<?>[] classes = new Class<?>[classnames.length];
+      for(int i = 0; i < classnames.length; i++) {
+        classes[i] = getClassByName(classnames[i]);
+      }
+      return classes;
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
   /** 
   /** 
    * Get the value of the <code>name</code> property as a <code>Class</code>.  
    * Get the value of the <code>name</code> property as a <code>Class</code>.  
    * If no such property is specified, then <code>defaultValue</code> is 
    * If no such property is specified, then <code>defaultValue</code> is 

+ 119 - 20
src/core/org/apache/hadoop/http/HttpServer.java

@@ -19,8 +19,12 @@ package org.apache.hadoop.http;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
-import java.net.URL;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
 import javax.servlet.ServletException;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
@@ -28,14 +32,17 @@ import javax.servlet.http.HttpServletResponse;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.log.LogLevel;
 import org.apache.hadoop.log.LogLevel;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
-
 import org.mortbay.http.HttpContext;
 import org.mortbay.http.HttpContext;
 import org.mortbay.http.SocketListener;
 import org.mortbay.http.SocketListener;
 import org.mortbay.http.SslListener;
 import org.mortbay.http.SslListener;
 import org.mortbay.http.handler.ResourceHandler;
 import org.mortbay.http.handler.ResourceHandler;
+import org.mortbay.jetty.servlet.Dispatcher;
+import org.mortbay.jetty.servlet.FilterHolder;
 import org.mortbay.jetty.servlet.WebApplicationContext;
 import org.mortbay.jetty.servlet.WebApplicationContext;
+import org.mortbay.jetty.servlet.WebApplicationHandler;
 
 
 /**
 /**
  * Create a Jetty embedded server to answer http requests. The primary goal
  * Create a Jetty embedded server to answer http requests. The primary goal
@@ -45,14 +52,24 @@ import org.mortbay.jetty.servlet.WebApplicationContext;
  *   "/static/" -> points to common static files (src/webapps/static)
  *   "/static/" -> points to common static files (src/webapps/static)
  *   "/" -> the jsp server code from (src/webapps/<name>)
  *   "/" -> the jsp server code from (src/webapps/<name>)
  */
  */
-public class HttpServer {
+public class HttpServer implements FilterContainer {
   public static final Log LOG = LogFactory.getLog(HttpServer.class);
   public static final Log LOG = LogFactory.getLog(HttpServer.class);
 
 
+  static final String FILTER_INITIALIZER_PROPERTY
+      = "hadoop.http.filter.initializers";
+
   protected final org.mortbay.jetty.Server webServer;
   protected final org.mortbay.jetty.Server webServer;
   protected final WebApplicationContext webAppContext;
   protected final WebApplicationContext webAppContext;
   protected final boolean findPort;
   protected final boolean findPort;
   protected final SocketListener listener;
   protected final SocketListener listener;
   private SslListener sslListener;
   private SslListener sslListener;
+  protected final List<String> filterNames = new ArrayList<String>();
+
+  /** Same as this(name, bindAddress, port, findPort, null); */
+  public HttpServer(String name, String bindAddress, int port, boolean findPort
+      ) throws IOException {
+    this(name, bindAddress, port, findPort, null);
+  }
 
 
   /**
   /**
    * Create a status server on the given port.
    * Create a status server on the given port.
@@ -61,9 +78,10 @@ public class HttpServer {
    * @param port The port to use on the server
    * @param port The port to use on the server
    * @param findPort whether the server should start at the given port and 
    * @param findPort whether the server should start at the given port and 
    *        increment by 1 until it finds a free port.
    *        increment by 1 until it finds a free port.
+   * @param conf Configuration 
    */
    */
-  public HttpServer(String name, String bindAddress, int port, boolean findPort
-      ) throws IOException {
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf) throws IOException {
     webServer = new org.mortbay.jetty.Server();
     webServer = new org.mortbay.jetty.Server();
     this.findPort = findPort;
     this.findPort = findPort;
     listener = new SocketListener();
     listener = new SocketListener();
@@ -73,9 +91,35 @@ public class HttpServer {
 
 
     final String appDir = getWebAppsPath();
     final String appDir = getWebAppsPath();
     webAppContext = webServer.addWebApplication("/", appDir + "/" + name);
     webAppContext = webServer.addWebApplication("/", appDir + "/" + name);
+
+    final FilterInitializer[] initializers = getFilterInitializers(conf); 
+    if (initializers != null) {
+      for(FilterInitializer c : initializers) {
+        c.initFilter(this);
+      }
+    }
     addWebapps(appDir);
     addWebapps(appDir);
   }
   }
 
 
+  /** Get an array of FilterConfiguration specified in the conf */
+  private static FilterInitializer[] getFilterInitializers(Configuration conf) {
+    if (conf == null) {
+      return null;
+    }
+
+    Class<?>[] classes = conf.getClasses(FILTER_INITIALIZER_PROPERTY);
+    if (classes == null) {
+      return null;
+    }
+
+    FilterInitializer[] initializers = new FilterInitializer[classes.length];
+    for(int i = 0; i < classes.length; i++) {
+      initializers[i] = (FilterInitializer)ReflectionUtils.newInstance(
+          classes[i], conf);
+    }
+    return initializers;
+  }
+
   /**
   /**
    * Add webapps and servlets.
    * Add webapps and servlets.
    * @param appDir The application directory
    * @param appDir The application directory
@@ -85,25 +129,34 @@ public class HttpServer {
     // 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");
     String logDir = System.getProperty("hadoop.log.dir");
     if (logDir != null) {
     if (logDir != null) {
-      HttpContext logContext = new HttpContext();
-      logContext.setContextPath("/logs/*");
-      logContext.setResourceBase(logDir);
-      logContext.addHandler(new ResourceHandler());
-      webServer.addContext(logContext);
+      addContext("/logs/*", logDir, true);
     }
     }
 
 
     // set up the context for "/static/*"
     // set up the context for "/static/*"
-    HttpContext staticContext = new HttpContext();
-    staticContext.setContextPath("/static/*");
-    staticContext.setResourceBase(appDir + "/static");
-    staticContext.addHandler(new ResourceHandler());
-    webServer.addContext(staticContext);
+    addContext("/static/*", appDir + "/static", true);
 
 
     // set up default servlets
     // set up default servlets
     addServlet("stacks", "/stacks", StackServlet.class);
     addServlet("stacks", "/stacks", StackServlet.class);
     addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
     addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
   }
   }
 
 
+  /**
+   * 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 
+   */
+  protected void addContext(String pathSpec, String dir, boolean isFiltered) {
+    HttpContext context = new HttpContext();
+    context.setContextPath(pathSpec);
+    context.setResourceBase(dir);
+    context.addHandler(new ResourceHandler());
+    webServer.addContext(context);
+    if (isFiltered) {
+      addFilterPathMapping(pathSpec);
+    }
+  }
+
   /**
   /**
    * Set a value in the webapp context. These values are available to the jsp
    * Set a value in the webapp context. These values are available to the jsp
    * pages as "application.getAttribute(name)".
    * pages as "application.getAttribute(name)".
@@ -118,15 +171,29 @@ public class HttpServer {
    * Add a servlet in the server.
    * Add a servlet in the server.
    * @param name The name of the servlet (can be passed as null)
    * @param name The name of the servlet (can be passed as null)
    * @param pathSpec The path spec for the servlet
    * @param pathSpec The path spec for the servlet
-   * @param servletClass The servlet class
+   * @param clazz The servlet class
    */
    */
-  public <T extends HttpServlet> void addServlet(String name, String pathSpec,
-      Class<T> servletClass) {
+  public void addServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz) {
+    addInternalServlet(name, pathSpec, clazz);
+    addFilterPathMapping(pathSpec);
+  }
+
+  /**
+   * Add an internal servlet in the server.
+   * @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
+   * @deprecated this is a temporary method
+   */
+  @Deprecated
+  public void addInternalServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz) {
     try {
     try {
       if (name == null) {
       if (name == null) {
-        webAppContext.addServlet(pathSpec, servletClass.getName());
+        webAppContext.addServlet(pathSpec, clazz.getName());
       } else {
       } else {
-        webAppContext.addServlet(name, pathSpec, servletClass.getName());
+        webAppContext.addServlet(name, pathSpec, clazz.getName());
       } 
       } 
     } catch (ClassNotFoundException cnfe) {
     } catch (ClassNotFoundException cnfe) {
       throw new RuntimeException("Problem instantiating class", cnfe);
       throw new RuntimeException("Problem instantiating class", cnfe);
@@ -137,6 +204,38 @@ public class HttpServer {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
+  public void addFilter(String name, String classname,
+      Map<String, String> parameters) {
+    WebApplicationHandler handler = webAppContext.getWebApplicationHandler();
+
+    LOG.info("adding " + name + " (class=" + classname + ")");
+    filterNames.add(name);
+
+    FilterHolder holder = handler.defineFilter(name, classname);
+    if (parameters != null) {
+      for(Map.Entry<String, String> e : parameters.entrySet()) {
+        holder.setInitParameter(e.getKey(), e.getValue());
+      }
+    }
+
+    final String[] USER_FACING_URLS = {"*.html", "*.jsp"};
+    for(String url : USER_FACING_URLS) {
+      handler.addFilterPathMapping(url, name, Dispatcher.__ALL);
+    }
+  }
+
+  /**
+   * Add the path spec to the filter path mapping.
+   * @param pathSpec The path spec
+   */
+  protected void addFilterPathMapping(String pathSpec) {
+    WebApplicationHandler handler = webAppContext.getWebApplicationHandler();
+    for(String name : filterNames) {
+      handler.addFilterPathMapping(pathSpec, name, Dispatcher.__ALL);
+    }
+  }
+
   /**
   /**
    * Get the value in the webapp context.
    * Get the value in the webapp context.
    * @param name The name of the attribute
    * @param name The name of the attribute

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

@@ -347,7 +347,8 @@ public class DataNode extends Configured
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     String infoHost = infoSocAddr.getHostName();
     String infoHost = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
     int tmpInfoPort = infoSocAddr.getPort();
-    this.infoServer = new HttpServer("datanode", infoHost, tmpInfoPort, tmpInfoPort == 0);
+    this.infoServer = new HttpServer("datanode", infoHost, tmpInfoPort,
+        tmpInfoPort == 0, conf);
     InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(
     InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(
         conf.get("dfs.datanode.https.address", infoHost + ":" + 0));
         conf.get("dfs.datanode.https.address", infoHost + ":" + 0));
     Configuration sslConf = new Configuration(conf);
     Configuration sslConf = new Configuration(conf);
@@ -358,7 +359,7 @@ public class DataNode extends Configured
           sslConf.get("https.keystore.password", ""),
           sslConf.get("https.keystore.password", ""),
           sslConf.get("https.keystore.keypassword", ""));
           sslConf.get("https.keystore.keypassword", ""));
     }
     }
-    this.infoServer.addServlet(null, "/streamFile/*", StreamFile.class);
+    this.infoServer.addInternalServlet(null, "/streamFile/*", StreamFile.class);
     this.infoServer.setAttribute("datanode.blockScanner", blockScanner);
     this.infoServer.setAttribute("datanode.blockScanner", blockScanner);
     this.infoServer.addServlet(null, "/blockScannerReport", 
     this.infoServer.addServlet(null, "/blockScannerReport", 
                                DataBlockScanner.Servlet.class);
                                DataBlockScanner.Servlet.class);

+ 5 - 5
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -344,7 +344,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     String infoHost = infoSocAddr.getHostName();
     String infoHost = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
     int tmpInfoPort = infoSocAddr.getPort();
     this.infoServer = new HttpServer("hdfs", infoHost, tmpInfoPort, 
     this.infoServer = new HttpServer("hdfs", infoHost, tmpInfoPort, 
-                                            tmpInfoPort == 0);
+        tmpInfoPort == 0, conf);
     InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(
     InetSocketAddress secInfoSocAddr = NetUtils.createSocketAddr(
         conf.get("dfs.https.address", infoHost + ":" + 0));
         conf.get("dfs.https.address", infoHost + ":" + 0));
     Configuration sslConf = new Configuration(conf);
     Configuration sslConf = new Configuration(conf);
@@ -363,10 +363,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
     this.infoServer.setAttribute("name.node", nn);
     this.infoServer.setAttribute("name.node", nn);
     this.infoServer.setAttribute("name.system.image", getFSImage());
     this.infoServer.setAttribute("name.system.image", getFSImage());
     this.infoServer.setAttribute("name.conf", conf);
     this.infoServer.setAttribute("name.conf", conf);
-    this.infoServer.addServlet("fsck", "/fsck", FsckServlet.class);
-    this.infoServer.addServlet("getimage", "/getimage", GetImageServlet.class);
-    this.infoServer.addServlet("listPaths", "/listPaths/*", ListPathsServlet.class);
-    this.infoServer.addServlet("data", "/data/*", FileDataServlet.class);
+    this.infoServer.addInternalServlet("fsck", "/fsck", FsckServlet.class);
+    this.infoServer.addInternalServlet("getimage", "/getimage", GetImageServlet.class);
+    this.infoServer.addInternalServlet("listPaths", "/listPaths/*", ListPathsServlet.class);
+    this.infoServer.addInternalServlet("data", "/data/*", FileDataServlet.class);
     this.infoServer.start();
     this.infoServer.start();
 
 
     // The web-server port can be ephemeral... ensure we have the correct info
     // The web-server port can be ephemeral... ensure we have the correct info

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

@@ -151,11 +151,11 @@ public class SecondaryNameNode implements FSConstants, Runnable {
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     infoBindAddress = infoSocAddr.getHostName();
     infoBindAddress = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
     int tmpInfoPort = infoSocAddr.getPort();
-    infoServer = new HttpServer("secondary", infoBindAddress, tmpInfoPort, 
-                                      tmpInfoPort == 0);
+    infoServer = new HttpServer("secondary", infoBindAddress, tmpInfoPort,
+        tmpInfoPort == 0, conf);
     infoServer.setAttribute("name.system.image", checkpointImage);
     infoServer.setAttribute("name.system.image", checkpointImage);
     this.infoServer.setAttribute("name.conf", conf);
     this.infoServer.setAttribute("name.conf", conf);
-    infoServer.addServlet("getimage", "/getimage", GetImageServlet.class);
+    infoServer.addInternalServlet("getimage", "/getimage", GetImageServlet.class);
     infoServer.start();
     infoServer.start();
 
 
     // The web-server port can be ephemeral... ensure we have the correct info
     // The web-server port can be ephemeral... ensure we have the correct info

+ 1 - 6
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -54,11 +54,6 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
 import org.apache.hadoop.ipc.RPC.VersionMismatch;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-import org.apache.hadoop.metrics.jvm.JvmMetrics;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
@@ -568,7 +563,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     String infoBindAddress = infoSocAddr.getHostName();
     String infoBindAddress = infoSocAddr.getHostName();
     int tmpInfoPort = infoSocAddr.getPort();
     int tmpInfoPort = infoSocAddr.getPort();
     infoServer = new StatusHttpServer("job", infoBindAddress, tmpInfoPort, 
     infoServer = new StatusHttpServer("job", infoBindAddress, tmpInfoPort, 
-                                      tmpInfoPort == 0);
+        tmpInfoPort == 0, conf);
     infoServer.setAttribute("job.tracker", this);
     infoServer.setAttribute("job.tracker", this);
     // initialize history parameters.
     // initialize history parameters.
     boolean historyInitialized = JobHistory.init(conf, this.localMachine);
     boolean historyInitialized = JobHistory.init(conf, this.localMachine);

+ 5 - 3
src/mapred/org/apache/hadoop/mapred/StatusHttpServer.java

@@ -25,6 +25,8 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponse;
 
 
+import org.apache.hadoop.conf.Configuration;
+
 /**
 /**
  * A mapred http server.  
  * A mapred http server.  
  */
  */
@@ -37,9 +39,9 @@ public class StatusHttpServer extends org.apache.hadoop.http.HttpServer {
    * @param findPort whether the server should start at the given port and 
    * @param findPort whether the server should start at the given port and 
    *        increment by 1 until it finds a free port.
    *        increment by 1 until it finds a free port.
    */
    */
-  StatusHttpServer(String name, String bindAddress, int port, 
-                          boolean findPort) throws IOException {
-    super(name, bindAddress, port, findPort);
+  StatusHttpServer(String name, String bindAddress, int port, boolean findPort,
+      Configuration conf) throws IOException {
+    super(name, bindAddress, port, findPort, conf);
     addServlet("reducegraph", "/taskgraph", TaskGraphServlet.class);
     addServlet("reducegraph", "/taskgraph", TaskGraphServlet.class);
   }
   }
 
 

+ 4 - 5
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.io.PrintStream;
 import java.io.RandomAccessFile;
 import java.io.RandomAccessFile;
-import java.lang.reflect.InvocationTargetException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
@@ -799,8 +798,8 @@ public class TaskTracker
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
     String httpBindAddress = infoSocAddr.getHostName();
     String httpBindAddress = infoSocAddr.getHostName();
     int httpPort = infoSocAddr.getPort();
     int httpPort = infoSocAddr.getPort();
-    this.server = new StatusHttpServer(
-                        "task", httpBindAddress, httpPort, httpPort == 0);
+    this.server = new StatusHttpServer("task", httpBindAddress, httpPort,
+        httpPort == 0, conf);
     workerThreads = conf.getInt("tasktracker.http.threads", 40);
     workerThreads = conf.getInt("tasktracker.http.threads", 40);
     this.shuffleServerMetrics = new ShuffleServerMetrics(conf);
     this.shuffleServerMetrics = new ShuffleServerMetrics(conf);
     server.setThreads(1, workerThreads);
     server.setThreads(1, workerThreads);
@@ -814,8 +813,8 @@ public class TaskTracker
     server.setAttribute("log", LOG);
     server.setAttribute("log", LOG);
     server.setAttribute("localDirAllocator", localDirAllocator);
     server.setAttribute("localDirAllocator", localDirAllocator);
     server.setAttribute("shuffleServerMetrics", shuffleServerMetrics);
     server.setAttribute("shuffleServerMetrics", shuffleServerMetrics);
-    server.addServlet("mapOutput", "/mapOutput", MapOutputServlet.class);
-    server.addServlet("taskLog", "/tasklog", TaskLogServlet.class);
+    server.addInternalServlet("mapOutput", "/mapOutput", MapOutputServlet.class);
+    server.addInternalServlet("taskLog", "/tasklog", TaskLogServlet.class);
     server.start();
     server.start();
     this.httpPort = server.getPort();
     this.httpPort = server.getPort();
     initialize();
     initialize();