You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2014/06/11 22:55:43 UTC

[3/4] HBASE-10336 Remove deprecated usage of Hadoop HttpServer in InfoServer (Eric Charles)

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/conf/ConfServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/conf/ConfServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/conf/ConfServlet.java
new file mode 100644
index 0000000..aee6231
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/conf/ConfServlet.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http.conf;
+
+import java.io.IOException;
+import java.io.Writer;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.http.HttpServer;
+
+/**
+ * A servlet to print out the running configuration data.
+ */
+@InterfaceAudience.LimitedPrivate({"HBase"})
+@InterfaceStability.Unstable
+public class ConfServlet extends HttpServlet {
+  private static final long serialVersionUID = 1L;
+
+  private static final String FORMAT_JSON = "json";
+  private static final String FORMAT_XML = "xml";
+  private static final String FORMAT_PARAM = "format";
+
+  /**
+   * Return the Configuration of the daemon hosting this servlet.
+   * This is populated when the HttpServer starts.
+   */
+  private Configuration getConfFromContext() {
+    Configuration conf = (Configuration)getServletContext().getAttribute(
+        HttpServer.CONF_CONTEXT_ATTRIBUTE);
+    assert conf != null;
+    return conf;
+  }
+
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws ServletException, IOException {
+
+    if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
+                                                   request, response)) {
+      return;
+    }
+
+    String format = request.getParameter(FORMAT_PARAM);
+    if (null == format) {
+      format = FORMAT_XML;
+    }
+
+    if (FORMAT_XML.equals(format)) {
+      response.setContentType("text/xml; charset=utf-8");
+    } else if (FORMAT_JSON.equals(format)) {
+      response.setContentType("application/json; charset=utf-8");
+    }
+
+    Writer out = response.getWriter();
+    try {
+      writeResponse(getConfFromContext(), out, format);
+    } catch (BadFormatException bfe) {
+      response.sendError(HttpServletResponse.SC_BAD_REQUEST, bfe.getMessage());
+    }
+    out.close();
+  }
+
+  /**
+   * Guts of the servlet - extracted for easy testing.
+   */
+  static void writeResponse(Configuration conf, Writer out, String format)
+    throws IOException, BadFormatException {
+    if (FORMAT_JSON.equals(format)) {
+      Configuration.dumpConfiguration(conf, out);
+    } else if (FORMAT_XML.equals(format)) {
+      conf.writeXml(out);
+    } else {
+      throw new BadFormatException("Bad format: " + format);
+    }
+  }
+
+  public static class BadFormatException extends Exception {
+    private static final long serialVersionUID = 1L;
+
+    public BadFormatException(String msg) {
+      super(msg);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
new file mode 100644
index 0000000..d3812b6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
@@ -0,0 +1,423 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * 
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.http.jmx;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Array;
+import java.util.Iterator;
+import java.util.Set;
+
+import javax.management.AttributeNotFoundException;
+import javax.management.InstanceNotFoundException;
+import javax.management.IntrospectionException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import javax.management.RuntimeErrorException;
+import javax.management.RuntimeMBeanException;
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.CompositeType;
+import javax.management.openmbean.TabularData;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonGenerator;
+
+/*
+ * This servlet is based off of the JMXProxyServlet from Tomcat 7.0.14. It has
+ * been rewritten to be read only and to output in a JSON format so it is not
+ * really that close to the original.
+ */
+/**
+ * Provides Read only web access to JMX.
+ * <p>
+ * This servlet generally will be placed under the /jmx URL for each
+ * HttpServer.  It provides read only
+ * access to JMX metrics.  The optional <code>qry</code> parameter
+ * may be used to query only a subset of the JMX Beans.  This query
+ * functionality is provided through the
+ * {@link MBeanServer#queryNames(ObjectName, javax.management.QueryExp)}
+ * method.
+ * <p>
+ * For example <code>http://.../jmx?qry=Hadoop:*</code> will return
+ * all hadoop metrics exposed through JMX.
+ * <p>
+ * The optional <code>get</code> parameter is used to query an specific 
+ * attribute of a JMX bean.  The format of the URL is
+ * <code>http://.../jmx?get=MXBeanName::AttributeName<code>
+ * <p>
+ * For example 
+ * <code>
+ * http://../jmx?get=Hadoop:service=NameNode,name=NameNodeInfo::ClusterId
+ * </code> will return the cluster id of the namenode mxbean.
+ * <p>
+ * If the <code>qry</code> or the <code>get</code> parameter is not formatted 
+ * correctly then a 400 BAD REQUEST http response code will be returned. 
+ * <p>
+ * If a resouce such as a mbean or attribute can not be found, 
+ * a 404 SC_NOT_FOUND http response code will be returned. 
+ * <p>
+ * The return format is JSON and in the form
+ * <p>
+ *  <code><pre>
+ *  {
+ *    "beans" : [
+ *      {
+ *        "name":"bean-name"
+ *        ...
+ *      }
+ *    ]
+ *  }
+ *  </pre></code>
+ *  <p>
+ *  The servlet attempts to convert the the JMXBeans into JSON. Each
+ *  bean's attributes will be converted to a JSON object member.
+ *  
+ *  If the attribute is a boolean, a number, a string, or an array
+ *  it will be converted to the JSON equivalent. 
+ *  
+ *  If the value is a {@link CompositeData} then it will be converted
+ *  to a JSON object with the keys as the name of the JSON member and
+ *  the value is converted following these same rules.
+ *  
+ *  If the value is a {@link TabularData} then it will be converted
+ *  to an array of the {@link CompositeData} elements that it contains.
+ *  
+ *  All other objects will be converted to a string and output as such.
+ *  
+ *  The bean's name and modelerType will be returned for all beans.
+ *
+ *  Optional paramater "callback" should be used to deliver JSONP response.
+ *  
+ */
+public class JMXJsonServlet extends HttpServlet {
+  private static final Log LOG = LogFactory.getLog(JMXJsonServlet.class);
+
+  private static final long serialVersionUID = 1L;
+
+  private static final String CALLBACK_PARAM = "callback";
+
+  /**
+   * MBean server.
+   */
+  protected transient MBeanServer mBeanServer;
+
+  /**
+   * Json Factory to create Json generators for write objects in json format
+   */
+  protected transient JsonFactory jsonFactory;
+  /**
+   * Initialize this servlet.
+   */
+  @Override
+  public void init() throws ServletException {
+    // Retrieve the MBean server
+    mBeanServer = ManagementFactory.getPlatformMBeanServer();
+    jsonFactory = new JsonFactory();
+  }
+
+  /**
+   * Process a GET request for the specified resource.
+   * 
+   * @param request
+   *          The servlet request we are processing
+   * @param response
+   *          The servlet response we are creating
+   */
+  @Override
+  public void doGet(HttpServletRequest request, HttpServletResponse response) {
+    try {
+      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
+                                                     request, response)) {
+        return;
+      }
+      JsonGenerator jg = null;
+      String jsonpcb = null;
+      PrintWriter writer = null;
+      try {
+        writer = response.getWriter();
+ 
+        // "callback" parameter implies JSONP outpout
+        jsonpcb = request.getParameter(CALLBACK_PARAM);
+        if (jsonpcb != null) {
+          response.setContentType("application/javascript; charset=utf8");
+          writer.write(jsonpcb + "(");
+        } else {
+          response.setContentType("application/json; charset=utf8");
+        }
+
+        jg = jsonFactory.createJsonGenerator(writer);
+        jg.disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET);
+        jg.useDefaultPrettyPrinter();
+        jg.writeStartObject();
+
+        // query per mbean attribute
+        String getmethod = request.getParameter("get");
+        if (getmethod != null) {
+          String[] splitStrings = getmethod.split("\\:\\:");
+          if (splitStrings.length != 2) {
+            jg.writeStringField("result", "ERROR");
+            jg.writeStringField("message", "query format is not as expected.");
+            jg.flush();
+            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+            return;
+          }
+          listBeans(jg, new ObjectName(splitStrings[0]), splitStrings[1],
+              response);
+          return;
+        }
+
+        // query per mbean
+        String qry = request.getParameter("qry");
+        if (qry == null) {
+          qry = "*:*";
+        }
+        listBeans(jg, new ObjectName(qry), null, response);
+      } finally {
+        if (jg != null) {
+          jg.close();
+        }
+        if (jsonpcb != null) {
+           writer.write(");");
+        }
+        if (writer != null) {
+          writer.close();
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Caught an exception while processing JMX request", e);
+      response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } catch (MalformedObjectNameException e) {
+      LOG.error("Caught an exception while processing JMX request", e);
+      response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+    }
+  }
+
+  // --------------------------------------------------------- Private Methods
+  private void listBeans(JsonGenerator jg, ObjectName qry, String attribute, 
+      HttpServletResponse response) 
+  throws IOException {
+    LOG.debug("Listing beans for "+qry);
+    Set<ObjectName> names = null;
+    names = mBeanServer.queryNames(qry, null);
+
+    jg.writeArrayFieldStart("beans");
+    Iterator<ObjectName> it = names.iterator();
+    while (it.hasNext()) {
+      ObjectName oname = it.next();
+      MBeanInfo minfo;
+      String code = "";
+      Object attributeinfo = null;
+      try {
+        minfo = mBeanServer.getMBeanInfo(oname);
+        code = minfo.getClassName();
+        String prs = "";
+        try {
+          if ("org.apache.commons.modeler.BaseModelMBean".equals(code)) {
+            prs = "modelerType";
+            code = (String) mBeanServer.getAttribute(oname, prs);
+          }
+          if (attribute!=null) {
+            prs = attribute;
+            attributeinfo = mBeanServer.getAttribute(oname, prs);
+          }
+        } catch (AttributeNotFoundException e) {
+          // If the modelerType attribute was not found, the class name is used
+          // instead.
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch (MBeanException e) {
+          // The code inside the attribute getter threw an exception so log it,
+          // and fall back on the class name
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch (RuntimeException e) {
+          // For some reason even with an MBeanException available to them
+          // Runtime exceptionscan still find their way through, so treat them
+          // the same as MBeanException
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        } catch ( ReflectionException e ) {
+          // This happens when the code inside the JMX bean (setter?? from the
+          // java docs) threw an exception, so log it and fall back on the 
+          // class name
+          LOG.error("getting attribute " + prs + " of " + oname
+              + " threw an exception", e);
+        }
+      } catch (InstanceNotFoundException e) {
+        //Ignored for some reason the bean was not found so don't output it
+        continue;
+      } catch ( IntrospectionException e ) {
+        // This is an internal error, something odd happened with reflection so
+        // log it and don't output the bean.
+        LOG.error("Problem while trying to process JMX query: " + qry
+            + " with MBean " + oname, e);
+        continue;
+      } catch ( ReflectionException e ) {
+        // This happens when the code inside the JMX bean threw an exception, so
+        // log it and don't output the bean.
+        LOG.error("Problem while trying to process JMX query: " + qry
+            + " with MBean " + oname, e);
+        continue;
+      }
+
+      jg.writeStartObject();
+      jg.writeStringField("name", oname.toString());
+      
+      jg.writeStringField("modelerType", code);
+      if ((attribute != null) && (attributeinfo == null)) {
+        jg.writeStringField("result", "ERROR");
+        jg.writeStringField("message", "No attribute with name " + attribute
+            + " was found.");
+        jg.writeEndObject();
+        jg.writeEndArray();
+        jg.close();
+        response.setStatus(HttpServletResponse.SC_NOT_FOUND);
+        return;
+      }
+      
+      if (attribute != null) {
+        writeAttribute(jg, attribute, attributeinfo);
+      } else {
+        MBeanAttributeInfo attrs[] = minfo.getAttributes();
+        for (int i = 0; i < attrs.length; i++) {
+          writeAttribute(jg, oname, attrs[i]);
+        }
+      }
+      jg.writeEndObject();
+    }
+    jg.writeEndArray();
+  }
+
+  private void writeAttribute(JsonGenerator jg, ObjectName oname, MBeanAttributeInfo attr) throws IOException {
+    if (!attr.isReadable()) {
+      return;
+    }
+    String attName = attr.getName();
+    if ("modelerType".equals(attName)) {
+      return;
+    }
+    if (attName.indexOf("=") >= 0 || attName.indexOf(":") >= 0
+        || attName.indexOf(" ") >= 0) {
+      return;
+    }
+    Object value = null;
+    try {
+      value = mBeanServer.getAttribute(oname, attName);
+    } catch (RuntimeMBeanException e) {
+      // UnsupportedOperationExceptions happen in the normal course of business,
+      // so no need to log them as errors all the time.
+      if (e.getCause() instanceof UnsupportedOperationException) {
+        LOG.debug("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      } else {
+        LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      }
+      return;
+    } catch (RuntimeErrorException e) {
+      // RuntimeErrorException happens when an unexpected failure occurs in getAttribute
+      // for example https://issues.apache.org/jira/browse/DAEMON-120
+      LOG.debug("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (AttributeNotFoundException e) {
+      //Ignored the attribute was not found, which should never happen because the bean
+      //just told us that it has this attribute, but if this happens just don't output
+      //the attribute.
+      return;
+    } catch (MBeanException e) {
+      //The code inside the attribute getter threw an exception so log it, and
+      // skip outputting the attribute
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (RuntimeException e) {
+      //For some reason even with an MBeanException available to them Runtime exceptions
+      //can still find their way through, so treat them the same as MBeanException
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (ReflectionException e) {
+      //This happens when the code inside the JMX bean (setter?? from the java docs)
+      //threw an exception, so log it and skip outputting the attribute
+      LOG.error("getting attribute "+attName+" of "+oname+" threw an exception", e);
+      return;
+    } catch (InstanceNotFoundException e) {
+      //Ignored the mbean itself was not found, which should never happen because we
+      //just accessed it (perhaps something unregistered in-between) but if this
+      //happens just don't output the attribute.
+      return;
+    }
+
+    writeAttribute(jg, attName, value);
+  }
+  
+  private void writeAttribute(JsonGenerator jg, String attName, Object value) throws IOException {
+    jg.writeFieldName(attName);
+    writeObject(jg, value);
+  }
+  
+  private void writeObject(JsonGenerator jg, Object value) throws IOException {
+    if(value == null) {
+      jg.writeNull();
+    } else {
+      Class<?> c = value.getClass();
+      if (c.isArray()) {
+        jg.writeStartArray();
+        int len = Array.getLength(value);
+        for (int j = 0; j < len; j++) {
+          Object item = Array.get(value, j);
+          writeObject(jg, item);
+        }
+        jg.writeEndArray();
+      } else if(value instanceof Number) {
+        Number n = (Number)value;
+        jg.writeNumber(n.toString());
+      } else if(value instanceof Boolean) {
+        Boolean b = (Boolean)value;
+        jg.writeBoolean(b);
+      } else if(value instanceof CompositeData) {
+        CompositeData cds = (CompositeData)value;
+        CompositeType comp = cds.getCompositeType();
+        Set<String> keys = comp.keySet();
+        jg.writeStartObject();
+        for(String key: keys) {
+          writeAttribute(jg, key, cds.get(key));
+        }
+        jg.writeEndObject();
+      } else if(value instanceof TabularData) {
+        TabularData tds = (TabularData)value;
+        jg.writeStartArray();
+        for(Object entry : tds.values()) {
+          writeObject(jg, entry);
+        }
+        jg.writeEndArray();
+      } else {
+        jg.writeString(value.toString());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/package-info.java
new file mode 100644
index 0000000..c33b340
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * 
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * This package provides access to JMX primarily through the
+ * {@link org.apache.hadoop.hbase.http.jmx.JMXJsonServlet} class.
+ * <p>
+ * Copied from hadoop source code.<br/>
+ * See https://issues.apache.org/jira/browse/HADOOP-10232 to know why.
+ * </p>
+ */
+package org.apache.hadoop.hbase.http.jmx;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java
new file mode 100644
index 0000000..9e2f157
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http.lib;
+
+import java.io.IOException;
+import java.security.Principal;
+import java.util.HashMap;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.http.FilterContainer;
+import org.apache.hadoop.hbase.http.FilterInitializer;
+
+import javax.servlet.Filter;
+
+import static org.apache.hadoop.hbase.http.ServerConfigurationKeys.HBASE_HTTP_STATIC_USER;
+import static org.apache.hadoop.hbase.http.ServerConfigurationKeys.DEFAULT_HBASE_HTTP_STATIC_USER;
+
+/**
+ * Provides a servlet filter that pretends to authenticate a fake user (Dr.Who)
+ * so that the web UI is usable for a secure cluster without authentication.
+ */
+public class StaticUserWebFilter extends FilterInitializer {
+  static final String DEPRECATED_UGI_KEY = "dfs.web.ugi";
+
+  private static final Log LOG = LogFactory.getLog(StaticUserWebFilter.class);
+
+  static class User implements Principal {
+    private final String name;
+    public User(String name) {
+      this.name = name;
+    }
+    @Override
+    public String getName() {
+      return name;
+    }
+    @Override
+    public int hashCode() {
+      return name.hashCode();
+    }
+    @Override
+    public boolean equals(Object other) {
+      if (other == this) {
+        return true;
+      } else if (other == null || other.getClass() != getClass()) {
+        return false;
+      }
+      return ((User) other).name.equals(name);
+    }
+    @Override
+    public String toString() {
+      return name;
+    }    
+  }
+
+  public static class StaticUserFilter implements Filter {
+    private User user;
+    private String username;
+
+    @Override
+    public void destroy() {
+      // NOTHING
+    }
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response,
+                         FilterChain chain
+                         ) throws IOException, ServletException {
+      HttpServletRequest httpRequest = (HttpServletRequest) request;
+      // if the user is already authenticated, don't override it
+      if (httpRequest.getRemoteUser() != null) {
+        chain.doFilter(request, response);
+      } else {
+        HttpServletRequestWrapper wrapper = 
+            new HttpServletRequestWrapper(httpRequest) {
+          @Override
+          public Principal getUserPrincipal() {
+            return user;
+          }
+          @Override
+          public String getRemoteUser() {
+            return username;
+          }
+        };
+        chain.doFilter(wrapper, response);
+      }
+    }
+
+    @Override
+    public void init(FilterConfig conf) throws ServletException {
+      this.username = conf.getInitParameter(HBASE_HTTP_STATIC_USER);
+      this.user = new User(username);
+    }
+    
+  }
+
+  @Override
+  public void initFilter(FilterContainer container, Configuration conf) {
+    HashMap<String, String> options = new HashMap<String, String>();
+    
+    String username = getUsernameFromConf(conf);
+    options.put(HBASE_HTTP_STATIC_USER, username);
+
+    container.addFilter("static_user_filter", 
+                        StaticUserFilter.class.getName(), 
+                        options);
+  }
+
+  /**
+   * Retrieve the static username from the configuration.
+   */
+  static String getUsernameFromConf(Configuration conf) {
+    String oldStyleUgi = conf.get(DEPRECATED_UGI_KEY);
+    if (oldStyleUgi != null) {
+      // We can't use the normal configuration deprecation mechanism here
+      // since we need to split out the username from the configured UGI.
+      LOG.warn(DEPRECATED_UGI_KEY + " should not be used. Instead, use " + 
+          HBASE_HTTP_STATIC_USER + ".");
+      String[] parts = oldStyleUgi.split(",");
+      return parts[0];
+    } else {
+      return conf.get(HBASE_HTTP_STATIC_USER,
+        DEFAULT_HBASE_HTTP_STATIC_USER);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/package-info.java
new file mode 100644
index 0000000..f92fa57
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/package-info.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * <p>
+ * This package provides user-selectable (via configuration) classes that add
+ * functionality to the web UI. They are configured as a list of classes in the
+ * configuration parameter <b>hadoop.http.filter.initializers</b>.
+ * </p>
+ * <ul>
+ * <li> <b>StaticUserWebFilter</b> - An authorization plugin that makes all
+ * users a static configured user.
+ * </ul>
+ * <p>
+ * Copied from hadoop source code.<br/>
+ * See https://issues.apache.org/jira/browse/HADOOP-10232 to know why
+ * </p>
+ */
+@InterfaceAudience.LimitedPrivate({"HBase"})
+@InterfaceStability.Unstable
+package org.apache.hadoop.hbase.http.lib;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
new file mode 100644
index 0000000..c2a47c0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http.log;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.regex.Pattern;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Jdk14Logger;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.apache.hadoop.util.ServletUtil;
+
+/**
+ * Change log level in runtime.
+ */
+@InterfaceStability.Evolving
+public class LogLevel {
+  public static final String USAGES = "\nUsage: General options are:\n"
+      + "\t[-getlevel <host:httpPort> <name>]\n"
+      + "\t[-setlevel <host:httpPort> <name> <level>]\n";
+
+  /**
+   * A command line implementation
+   */
+  public static void main(String[] args) {
+    if (args.length == 3 && "-getlevel".equals(args[0])) {
+      process("http://" + args[1] + "/logLevel?log=" + args[2]);
+      return;
+    }
+    else if (args.length == 4 && "-setlevel".equals(args[0])) {
+      process("http://" + args[1] + "/logLevel?log=" + args[2]
+              + "&level=" + args[3]);
+      return;
+    }
+
+    System.err.println(USAGES);
+    System.exit(-1);
+  }
+
+  private static void process(String urlstring) {
+    try {
+      URL url = new URL(urlstring);
+      System.out.println("Connecting to " + url);
+      URLConnection connection = url.openConnection();
+      connection.connect();
+
+      BufferedReader in = new BufferedReader(new InputStreamReader(
+          connection.getInputStream()));
+      for(String line; (line = in.readLine()) != null; )
+        if (line.startsWith(MARKER)) {
+          System.out.println(TAG.matcher(line).replaceAll(""));
+        }
+      in.close();
+    } catch (IOException ioe) {
+      System.err.println("" + ioe);
+    }
+  }
+
+  static final String MARKER = "<!-- OUTPUT -->";
+  static final Pattern TAG = Pattern.compile("<[^>]*>");
+
+  /**
+   * A servlet implementation
+   */
+  @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+  @InterfaceStability.Unstable
+  public static class Servlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response
+        ) throws ServletException, IOException {
+
+      // Do the authorization
+      if (!HttpServer.hasAdministratorAccess(getServletContext(), request,
+          response)) {
+        return;
+      }
+
+      PrintWriter out = ServletUtil.initHTML(response, "Log Level");
+      String logName = ServletUtil.getParameter(request, "log");
+      String level = ServletUtil.getParameter(request, "level");
+
+      if (logName != null) {
+        out.println("<br /><hr /><h3>Results</h3>");
+        out.println(MARKER
+            + "Submitted Log Name: <b>" + logName + "</b><br />");
+
+        Log log = LogFactory.getLog(logName);
+        out.println(MARKER
+            + "Log Class: <b>" + log.getClass().getName() +"</b><br />");
+        if (level != null) {
+          out.println(MARKER + "Submitted Level: <b>" + level + "</b><br />");
+        }
+
+        if (log instanceof Log4JLogger) {
+          process(((Log4JLogger)log).getLogger(), level, out);
+        }
+        else if (log instanceof Jdk14Logger) {
+          process(((Jdk14Logger)log).getLogger(), level, out);
+        }
+        else {
+          out.println("Sorry, " + log.getClass() + " not supported.<br />");
+        }
+      }
+
+      out.println(FORMS);
+      out.println(ServletUtil.HTML_TAIL);
+    }
+
+    static final String FORMS = "\n<br /><hr /><h3>Get / Set</h3>"
+        + "\n<form>Log: <input type='text' size='50' name='log' /> "
+        + "<input type='submit' value='Get Log Level' />"
+        + "</form>"
+        + "\n<form>Log: <input type='text' size='50' name='log' /> "
+        + "Level: <input type='text' name='level' /> "
+        + "<input type='submit' value='Set Log Level' />"
+        + "</form>";
+
+    private static void process(org.apache.log4j.Logger log, String level,
+        PrintWriter out) throws IOException {
+      if (level != null) {
+        if (!level.equals(org.apache.log4j.Level.toLevel(level).toString())) {
+          out.println(MARKER + "Bad level : <b>" + level + "</b><br />");
+        } else {
+          log.setLevel(org.apache.log4j.Level.toLevel(level));
+          out.println(MARKER + "Setting Level to " + level + " ...<br />");
+        }
+      }
+      out.println(MARKER
+          + "Effective level: <b>" + log.getEffectiveLevel() + "</b><br />");
+    }
+
+    private static void process(java.util.logging.Logger log, String level,
+        PrintWriter out) throws IOException {
+      if (level != null) {
+        log.setLevel(java.util.logging.Level.parse(level));
+        out.println(MARKER + "Setting Level to " + level + " ...<br />");
+      }
+
+      java.util.logging.Level lev;
+      for(; (lev = log.getLevel()) == null; log = log.getParent());
+      out.println(MARKER + "Effective level: <b>" + lev + "</b><br />");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/package-info.java
new file mode 100644
index 0000000..045bafe
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * </ul>
+ * <p>
+ * Copied from hadoop source code.<br/>
+ * See https://issues.apache.org/jira/browse/HADOOP-10232 to know why.
+ * </p>
+ */
+@InterfaceStability.Unstable
+package org.apache.hadoop.hbase.http;
+
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index c798e4b..6a5c77e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -73,6 +73,8 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.executor.ExecutorType;
+import org.apache.hadoop.hbase.http.InfoServer;
+import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
 import org.apache.hadoop.hbase.ipc.RequestContext;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index f251bb8..fdf2cd3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -124,7 +125,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HasThread;
-import org.apache.hadoop.hbase.util.InfoServer;
 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Sleeper;

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
index d2c4e9d..0701a06 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
@@ -34,10 +34,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.rest.filter.AuthFilter;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.HttpServerUtil;
-import org.apache.hadoop.hbase.util.InfoServer;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.net.DNS;

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java
new file mode 100644
index 0000000..6981c8a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java
@@ -0,0 +1,365 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Arrays;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
+import org.apache.log4j.Layout;
+import org.apache.log4j.Logger;
+import org.apache.log4j.WriterAppender;
+import org.junit.Assert;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Supplier;
+import com.google.common.collect.Sets;
+
+/**
+ * Test provides some very generic helpers which might be used across the tests
+ */
+public abstract class GenericTestUtils {
+
+  private static final AtomicInteger sequence = new AtomicInteger();
+
+  /**
+   * Extracts the name of the method where the invocation has happened
+   * @return String name of the invoking method
+   */
+  public static String getMethodName() {
+    return Thread.currentThread().getStackTrace()[2].getMethodName();
+  }
+
+  /**
+   * Generates a process-wide unique sequence number.
+   * @return an unique sequence number
+   */
+  public static int uniqueSequenceId() {
+    return sequence.incrementAndGet();
+  }
+  
+  /**
+   * Assert that a given file exists.
+   */
+  public static void assertExists(File f) {
+    Assert.assertTrue("File " + f + " should exist", f.exists());
+  }
+    
+  /**
+   * List all of the files in 'dir' that match the regex 'pattern'.
+   * Then check that this list is identical to 'expectedMatches'.
+   * @throws IOException if the dir is inaccessible
+   */
+  public static void assertGlobEquals(File dir, String pattern,
+      String ... expectedMatches) throws IOException {
+    
+    Set<String> found = Sets.newTreeSet();
+    for (File f : FileUtil.listFiles(dir)) {
+      if (f.getName().matches(pattern)) {
+        found.add(f.getName());
+      }
+    }
+    Set<String> expectedSet = Sets.newTreeSet(
+        Arrays.asList(expectedMatches));
+    Assert.assertEquals("Bad files matching " + pattern + " in " + dir,
+        Joiner.on(",").join(expectedSet),
+        Joiner.on(",").join(found));
+  }
+  
+  public static void assertExceptionContains(String string, Throwable t) {
+    String msg = t.getMessage();
+    Assert.assertTrue(
+        "Expected to find '" + string + "' but got unexpected exception:"
+        + StringUtils.stringifyException(t), msg.contains(string));
+  }  
+
+  public static void waitFor(Supplier<Boolean> check,
+      int checkEveryMillis, int waitForMillis)
+      throws TimeoutException, InterruptedException
+  {
+    long st = Time.now();
+    do {
+      boolean result = check.get();
+      if (result) {
+        return;
+      }
+      
+      Thread.sleep(checkEveryMillis);
+    } while (Time.now() - st < waitForMillis);
+    
+    throw new TimeoutException("Timed out waiting for condition. " +
+        "Thread diagnostics:\n" +
+        TimedOutTestsListener.buildThreadDiagnosticString());
+  }
+  
+  public static class LogCapturer {
+    private StringWriter sw = new StringWriter();
+    private WriterAppender appender;
+    private Logger logger;
+    
+    public static LogCapturer captureLogs(Log l) {
+      Logger logger = ((Log4JLogger)l).getLogger();
+      LogCapturer c = new LogCapturer(logger);
+      return c;
+    }
+    
+
+    private LogCapturer(Logger logger) {
+      this.logger = logger;
+      Layout layout = Logger.getRootLogger().getAppender("stdout").getLayout();
+      WriterAppender wa = new WriterAppender(layout, sw);
+      logger.addAppender(wa);
+    }
+    
+    public String getOutput() {
+      return sw.toString();
+    }
+    
+    public void stopCapturing() {
+      logger.removeAppender(appender);
+
+    }
+  }
+  
+  
+  /**
+   * Mockito answer helper that triggers one latch as soon as the
+   * method is called, then waits on another before continuing.
+   */
+  public static class DelayAnswer implements Answer<Object> {
+    private final Log LOG;
+    
+    private final CountDownLatch fireLatch = new CountDownLatch(1);
+    private final CountDownLatch waitLatch = new CountDownLatch(1);
+    private final CountDownLatch resultLatch = new CountDownLatch(1);
+    
+    private final AtomicInteger fireCounter = new AtomicInteger(0);
+    private final AtomicInteger resultCounter = new AtomicInteger(0);
+    
+    // Result fields set after proceed() is called.
+    private volatile Throwable thrown;
+    private volatile Object returnValue;
+    
+    public DelayAnswer(Log log) {
+      this.LOG = log;
+    }
+
+    /**
+     * Wait until the method is called.
+     */
+    public void waitForCall() throws InterruptedException {
+      fireLatch.await();
+    }
+  
+    /**
+     * Tell the method to proceed.
+     * This should only be called after waitForCall()
+     */
+    public void proceed() {
+      waitLatch.countDown();
+    }
+  
+    @Override
+    public Object answer(InvocationOnMock invocation) throws Throwable {
+      LOG.info("DelayAnswer firing fireLatch");
+      fireCounter.getAndIncrement();
+      fireLatch.countDown();
+      try {
+        LOG.info("DelayAnswer waiting on waitLatch");
+        waitLatch.await();
+        LOG.info("DelayAnswer delay complete");
+      } catch (InterruptedException ie) {
+        throw new IOException("Interrupted waiting on latch", ie);
+      }
+      return passThrough(invocation);
+    }
+
+    protected Object passThrough(InvocationOnMock invocation) throws Throwable {
+      try {
+        Object ret = invocation.callRealMethod();
+        returnValue = ret;
+        return ret;
+      } catch (Throwable t) {
+        thrown = t;
+        throw t;
+      } finally {
+        resultCounter.incrementAndGet();
+        resultLatch.countDown();
+      }
+    }
+    
+    /**
+     * After calling proceed(), this will wait until the call has
+     * completed and a result has been returned to the caller.
+     */
+    public void waitForResult() throws InterruptedException {
+      resultLatch.await();
+    }
+    
+    /**
+     * After the call has gone through, return any exception that
+     * was thrown, or null if no exception was thrown.
+     */
+    public Throwable getThrown() {
+      return thrown;
+    }
+    
+    /**
+     * After the call has gone through, return the call's return value,
+     * or null in case it was void or an exception was thrown.
+     */
+    public Object getReturnValue() {
+      return returnValue;
+    }
+    
+    public int getFireCount() {
+      return fireCounter.get();
+    }
+    
+    public int getResultCount() {
+      return resultCounter.get();
+    }
+  }
+  
+  /**
+   * An Answer implementation that simply forwards all calls through
+   * to a delegate.
+   * 
+   * This is useful as the default Answer for a mock object, to create
+   * something like a spy on an RPC proxy. For example:
+   * <code>
+   *    NamenodeProtocol origNNProxy = secondary.getNameNode();
+   *    NamenodeProtocol spyNNProxy = Mockito.mock(NameNodeProtocol.class,
+   *        new DelegateAnswer(origNNProxy);
+   *    doThrow(...).when(spyNNProxy).getBlockLocations(...);
+   *    ...
+   * </code>
+   */
+  public static class DelegateAnswer implements Answer<Object> { 
+    private final Object delegate;
+    private final Log log;
+    
+    public DelegateAnswer(Object delegate) {
+      this(null, delegate);
+    }
+    
+    public DelegateAnswer(Log log, Object delegate) {
+      this.log = log;
+      this.delegate = delegate;
+    }
+
+    @Override
+    public Object answer(InvocationOnMock invocation) throws Throwable {
+      try {
+        if (log != null) {
+          log.info("Call to " + invocation + " on " + delegate,
+              new Exception("TRACE"));
+        }
+        return invocation.getMethod().invoke(
+            delegate, invocation.getArguments());
+      } catch (InvocationTargetException ite) {
+        throw ite.getCause();
+      }
+    }
+  }
+
+  /**
+   * An Answer implementation which sleeps for a random number of milliseconds
+   * between 0 and a configurable value before delegating to the real
+   * implementation of the method. This can be useful for drawing out race
+   * conditions.
+   */
+  public static class SleepAnswer implements Answer<Object> {
+    private final int maxSleepTime;
+    private static Random r = new Random();
+    
+    public SleepAnswer(int maxSleepTime) {
+      this.maxSleepTime = maxSleepTime;
+    }
+    
+    @Override
+    public Object answer(InvocationOnMock invocation) throws Throwable {
+      boolean interrupted = false;
+      try {
+        Thread.sleep(r.nextInt(maxSleepTime));
+      } catch (InterruptedException ie) {
+        interrupted = true;
+      }
+      try {
+        return invocation.callRealMethod();
+      } finally {
+        if (interrupted) {
+          Thread.currentThread().interrupt();
+        }
+      }
+    }
+  }
+
+  public static void assertMatches(String output, String pattern) {
+    Assert.assertTrue("Expected output to match /" + pattern + "/" +
+        " but got:\n" + output,
+        Pattern.compile(pattern).matcher(output).find());
+  }
+  
+  public static void assertValueNear(long expected, long actual, long allowedError) {
+    assertValueWithinRange(expected - allowedError, expected + allowedError, actual);
+  }
+  
+  public static void assertValueWithinRange(long expectedMin, long expectedMax,
+      long actual) {
+    Assert.assertTrue("Expected " + actual + " to be in range (" + expectedMin + ","
+        + expectedMax + ")", expectedMin <= actual && actual <= expectedMax);
+  }
+
+  /**
+   * Assert that there are no threads running whose name matches the
+   * given regular expression.
+   * @param regex the regex to match against
+   */
+  public static void assertNoThreadsMatching(String regex) {
+    Pattern pattern = Pattern.compile(regex);
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+    
+    ThreadInfo[] infos = threadBean.getThreadInfo(threadBean.getAllThreadIds(), 20);
+    for (ThreadInfo info : infos) {
+      if (info == null) continue;
+      if (pattern.matcher(info.getThreadName()).matches()) {
+        Assert.fail("Leaked thread: " + info + "\n" +
+            Joiner.on("\n").join(info.getStackTrace()));
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/TimedOutTestsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TimedOutTestsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TimedOutTestsListener.java
new file mode 100644
index 0000000..9b3784d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TimedOutTestsListener.java
@@ -0,0 +1,175 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.management.LockInfo;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MonitorInfo;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+import org.junit.runner.notification.Failure;
+import org.junit.runner.notification.RunListener;
+
+/**
+ * JUnit run listener which prints full thread dump into System.err
+ * in case a test is failed due to timeout.
+ */
+public class TimedOutTestsListener extends RunListener {
+
+  static final String TEST_TIMED_OUT_PREFIX = "test timed out after";
+  
+  private static String INDENT = "    ";
+
+  private final PrintWriter output;
+  
+  public TimedOutTestsListener() {
+    this.output = new PrintWriter(System.err);
+  }
+  
+  public TimedOutTestsListener(PrintWriter output) {
+    this.output = output;
+  }
+
+  @Override
+  public void testFailure(Failure failure) throws Exception {
+    if (failure != null && failure.getMessage() != null 
+        && failure.getMessage().startsWith(TEST_TIMED_OUT_PREFIX)) {
+      output.println("====> TEST TIMED OUT. PRINTING THREAD DUMP. <====");
+      output.println();
+      output.print(buildThreadDiagnosticString());
+    }
+  }
+  
+  public static String buildThreadDiagnosticString() {
+    StringWriter sw = new StringWriter();
+    PrintWriter output = new PrintWriter(sw);
+    
+    DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd hh:mm:ss,SSS");
+    output.println(String.format("Timestamp: %s", dateFormat.format(new Date())));
+    output.println();
+    output.println(buildThreadDump());
+    
+    String deadlocksInfo = buildDeadlockInfo();
+    if (deadlocksInfo != null) {
+      output.println("====> DEADLOCKS DETECTED <====");
+      output.println();
+      output.println(deadlocksInfo);
+    }
+
+    return sw.toString();
+  }
+
+  static String buildThreadDump() {
+    StringBuilder dump = new StringBuilder();
+    Map<Thread, StackTraceElement[]> stackTraces = Thread.getAllStackTraces();
+    for (Map.Entry<Thread, StackTraceElement[]> e : stackTraces.entrySet()) {
+      Thread thread = e.getKey();
+      dump.append(String.format(
+          "\"%s\" %s prio=%d tid=%d %s\njava.lang.Thread.State: %s",
+          thread.getName(),
+          (thread.isDaemon() ? "daemon" : ""),
+          thread.getPriority(),
+          thread.getId(),
+          Thread.State.WAITING.equals(thread.getState()) ? 
+              "in Object.wait()" : thread.getState().name().toLowerCase(),
+          Thread.State.WAITING.equals(thread.getState()) ? 
+              "WAITING (on object monitor)" : thread.getState()));
+      for (StackTraceElement stackTraceElement : e.getValue()) {
+        dump.append("\n        at ");
+        dump.append(stackTraceElement);
+      }
+      dump.append("\n");
+    }
+    return dump.toString();
+  }
+  
+  static String buildDeadlockInfo() {
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+    long[] threadIds = threadBean.findMonitorDeadlockedThreads();
+    if (threadIds != null && threadIds.length > 0) {
+      StringWriter stringWriter = new StringWriter();
+      PrintWriter out = new PrintWriter(stringWriter);
+      
+      ThreadInfo[] infos = threadBean.getThreadInfo(threadIds, true, true);
+      for (ThreadInfo ti : infos) {
+        printThreadInfo(ti, out);
+        printLockInfo(ti.getLockedSynchronizers(), out);
+        out.println();
+      }
+      
+      out.close();
+      return stringWriter.toString();
+    } else {
+      return null;
+    }
+  }
+  
+  private static void printThreadInfo(ThreadInfo ti, PrintWriter out) {
+    // print thread information
+    printThread(ti, out);
+
+    // print stack trace with locks
+    StackTraceElement[] stacktrace = ti.getStackTrace();
+    MonitorInfo[] monitors = ti.getLockedMonitors();
+    for (int i = 0; i < stacktrace.length; i++) {
+      StackTraceElement ste = stacktrace[i];
+      out.println(INDENT + "at " + ste.toString());
+      for (MonitorInfo mi : monitors) {
+        if (mi.getLockedStackDepth() == i) {
+          out.println(INDENT + "  - locked " + mi);
+        }
+      }
+    }
+    out.println();
+  }
+
+  private static void printThread(ThreadInfo ti, PrintWriter out) {
+    out.print("\"" + ti.getThreadName() + "\"" + " Id="
+        + ti.getThreadId() + " in " + ti.getThreadState());
+    if (ti.getLockName() != null) {
+      out.print(" on lock=" + ti.getLockName());
+    }
+    if (ti.isSuspended()) {
+      out.print(" (suspended)");
+    }
+    if (ti.isInNative()) {
+      out.print(" (running in native)");
+    }
+    out.println();
+    if (ti.getLockOwnerName() != null) {
+      out.println(INDENT + " owned by " + ti.getLockOwnerName() + " Id="
+          + ti.getLockOwnerId());
+    }
+  }
+
+  private static void printLockInfo(LockInfo[] locks, PrintWriter out) {
+    out.println(INDENT + "Locked synchronizers: count = " + locks.length);
+    for (LockInfo li : locks) {
+      out.println(INDENT + "  - " + li);
+    }
+    out.println();
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
new file mode 100644
index 0000000..8844386
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
@@ -0,0 +1,232 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+package org.apache.hadoop.hbase.http;
+
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.junit.Assert;
+import org.junit.experimental.categories.Category;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.apache.hadoop.hbase.http.HttpServer.Builder;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URL;
+import java.net.MalformedURLException;
+
+/**
+ * This is a base class for functional tests of the {@link HttpServer}.
+ * The methods are static for other classes to import statically.
+ */
+public class HttpServerFunctionalTest extends Assert {
+  /** JVM property for the webapp test dir : {@value} */
+  public static final String TEST_BUILD_WEBAPPS = "test.build.webapps";
+  /** expected location of the test.build.webapps dir: {@value} */
+  private static final String BUILD_WEBAPPS_DIR = "build/test/webapps";
+  
+  /** name of the test webapp: {@value} */
+  private static final String TEST = "test";
+
+  /**
+   * Create but do not start the test webapp server. The test webapp dir is
+   * prepared/checked in advance.
+   *
+   * @return the server instance
+   *
+   * @throws IOException if a problem occurs
+   * @throws AssertionError if a condition was not met
+   */
+  public static HttpServer createTestServer() throws IOException {
+    prepareTestWebapp();
+    return createServer(TEST);
+  }
+
+  /**
+   * Create but do not start the test webapp server. The test webapp dir is
+   * prepared/checked in advance.
+   * @param conf the server configuration to use
+   * @return the server instance
+   *
+   * @throws IOException if a problem occurs
+   * @throws AssertionError if a condition was not met
+   */
+  public static HttpServer createTestServer(Configuration conf)
+      throws IOException {
+    prepareTestWebapp();
+    return createServer(TEST, conf);
+  }
+
+  public static HttpServer createTestServer(Configuration conf, AccessControlList adminsAcl)
+      throws IOException {
+    prepareTestWebapp();
+    return createServer(TEST, conf, adminsAcl);
+  }
+
+  /**
+   * Create but do not start the test webapp server. The test webapp dir is
+   * prepared/checked in advance.
+   * @param conf the server configuration to use
+   * @return the server instance
+   *
+   * @throws IOException if a problem occurs
+   * @throws AssertionError if a condition was not met
+   */
+  public static HttpServer createTestServer(Configuration conf, 
+      String[] pathSpecs) throws IOException {
+    prepareTestWebapp();
+    return createServer(TEST, conf, pathSpecs);
+  }
+
+  /**
+   * Prepare the test webapp by creating the directory from the test properties
+   * fail if the directory cannot be created.
+   * @throws AssertionError if a condition was not met
+   */
+  protected static void prepareTestWebapp() {
+    String webapps = System.getProperty(TEST_BUILD_WEBAPPS, BUILD_WEBAPPS_DIR);
+    File testWebappDir = new File(webapps +
+        File.separatorChar + TEST);
+    try {
+    if (!testWebappDir.exists()) {
+      fail("Test webapp dir " + testWebappDir.getCanonicalPath() + " missing");
+    }
+    }
+    catch (IOException e) {
+    }
+  }
+
+  /**
+   * Create an HttpServer instance on the given address for the given webapp
+   * @param host to bind
+   * @param port to bind
+   * @return the server
+   * @throws IOException if it could not be created
+   */
+  public static HttpServer createServer(String host, int port)
+      throws IOException {
+    prepareTestWebapp();
+    return new HttpServer.Builder().setName(TEST)
+        .addEndpoint(URI.create("http://" + host + ":" + port))
+        .setFindPort(true).build();
+  }
+
+  /**
+   * Create an HttpServer instance for the given webapp
+   * @param webapp the webapp to work with
+   * @return the server
+   * @throws IOException if it could not be created
+   */
+  public static HttpServer createServer(String webapp) throws IOException {
+    return localServerBuilder(webapp).setFindPort(true).build();
+  }
+  /**
+   * Create an HttpServer instance for the given webapp
+   * @param webapp the webapp to work with
+   * @param conf the configuration to use for the server
+   * @return the server
+   * @throws IOException if it could not be created
+   */
+  public static HttpServer 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)
+      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(
+        URI.create("http://localhost:0"));
+  }
+  
+  /**
+   * Create an HttpServer instance for the given webapp
+   * @param webapp the webapp to work with
+   * @param conf the configuration to use for the server
+   * @param pathSpecs the paths specifications the server will service
+   * @return the server
+   * @throws IOException if it could not be created
+   */
+  public static HttpServer createServer(String webapp, Configuration conf,
+      String[] pathSpecs) throws IOException {
+    return localServerBuilder(webapp).setFindPort(true).setConf(conf).setPathSpec(pathSpecs).build();
+  }
+
+  /**
+   * Create and start a server with the test webapp
+   *
+   * @return the newly started server
+   *
+   * @throws IOException on any failure
+   * @throws AssertionError if a condition was not met
+   */
+  public static HttpServer createAndStartTestServer() throws IOException {
+    HttpServer server = createTestServer();
+    server.start();
+    return server;
+  }
+
+  /**
+   * If the server is non null, stop it
+   * @param server to stop
+   * @throws Exception on any failure
+   */
+  public static void stop(HttpServer server) throws Exception {
+    if (server != null) {
+      server.stop();
+    }
+  }
+
+  /**
+   * Pass in a server, return a URL bound to localhost and its port
+   * @param server server
+   * @return a URL bonded to the base of the server
+   * @throws MalformedURLException if the URL cannot be created.
+   */
+  public static URL getServerURL(HttpServer server)
+      throws MalformedURLException {
+    assertNotNull("No server", server);
+    return new URL("http://"
+        + NetUtils.getHostPortString(server.getConnectorAddress(0)));
+  }
+
+  /**
+   * Read in the content from a URL
+   * @param url URL To read
+   * @return the text from the output
+   * @throws IOException if something went wrong
+   */
+  protected static String readOutput(URL url) throws IOException {
+    StringBuilder out = new StringBuilder();
+    InputStream in = url.openConnection().getInputStream();
+    byte[] buffer = new byte[64 * 1024];
+    int len = in.read(buffer);
+    while (len > 0) {
+      out.append(new String(buffer, 0, len));
+      len = in.read(buffer);
+    }
+    return out.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java
new file mode 100644
index 0000000..3f0260b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.URL;
+import java.net.URLConnection;
+import java.util.Set;
+import java.util.TreeSet;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.FilterContainer;
+import org.apache.hadoop.hbase.http.FilterInitializer;
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestGlobalFilter extends HttpServerFunctionalTest {
+  static final Log LOG = LogFactory.getLog(HttpServer.class);
+  static final Set<String> RECORDS = new TreeSet<String>(); 
+
+  /** A very simple filter that records accessed uri's */
+  static public class RecordingFilter implements Filter {
+    private FilterConfig filterConfig = null;
+
+    @Override
+    public void init(FilterConfig filterConfig) {
+      this.filterConfig = filterConfig;
+    }
+
+    @Override
+    public void destroy() {
+      this.filterConfig = null;
+    }
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response,
+        FilterChain chain) throws IOException, ServletException {
+      if (filterConfig == null)
+         return;
+
+      String uri = ((HttpServletRequest)request).getRequestURI();
+      LOG.info("filtering " + uri);
+      RECORDS.add(uri);
+      chain.doFilter(request, response);
+    }
+
+    /** Configuration for RecordingFilter */
+    static public class Initializer extends FilterInitializer {
+      public Initializer() {}
+
+      @Override
+      public void initFilter(FilterContainer container, Configuration conf) {
+        container.addGlobalFilter("recording", RecordingFilter.class.getName(), null);
+      }
+    }
+  }
+  
+  
+  /** access a url, ignoring some IOException such as the page does not exist */
+  static void access(String urlstring) throws IOException {
+    LOG.warn("access " + urlstring);
+    URL url = new URL(urlstring);
+    URLConnection connection = url.openConnection();
+    connection.connect();
+    
+    try {
+      BufferedReader in = new BufferedReader(new InputStreamReader(
+          connection.getInputStream()));
+      try {
+        for(; in.readLine() != null; );
+      } finally {
+        in.close();
+      }
+    } catch(IOException ioe) {
+      LOG.warn("urlstring=" + urlstring, ioe);
+    }
+  }
+
+  @Test
+  public void testServletFilter() throws Exception {
+    Configuration conf = new Configuration();
+    
+    //start a http server with CountingFilter
+    conf.set(HttpServer.FILTER_INITIALIZERS_PROPERTY,
+        RecordingFilter.Initializer.class.getName());
+    HttpServer http = createTestServer(conf);
+    http.start();
+
+    final String fsckURL = "/fsck";
+    final String stacksURL = "/stacks";
+    final String ajspURL = "/a.jsp";
+    final String listPathsURL = "/listPaths";
+    final String dataURL = "/data";
+    final String streamFile = "/streamFile";
+    final String rootURL = "/";
+    final String allURL = "/*";
+    final String outURL = "/static/a.out";
+    final String logURL = "/logs/a.log";
+
+    final String[] urls = {fsckURL, stacksURL, ajspURL, listPathsURL, 
+        dataURL, streamFile, rootURL, allURL, outURL, logURL};
+
+    //access the urls
+    final String prefix = "http://"
+        + NetUtils.getHostPortString(http.getConnectorAddress(0));
+    try {
+      for(int i = 0; i < urls.length; i++) {
+        access(prefix + urls[i]);
+      }
+    } finally {
+      http.stop();
+    }
+
+    LOG.info("RECORDS = " + RECORDS);
+    
+    //verify records
+    for(int i = 0; i < urls.length; i++) {
+      assertTrue(RECORDS.remove(urls[i]));
+    }
+    assertTrue(RECORDS.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java
new file mode 100644
index 0000000..969668c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;
+
+import static org.junit.Assert.*;
+
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.HtmlQuoting;
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+@Category(SmallTests.class)
+public class TestHtmlQuoting {
+
+  @Test public void testNeedsQuoting() throws Exception {
+    assertTrue(HtmlQuoting.needsQuoting("abcde>"));
+    assertTrue(HtmlQuoting.needsQuoting("<abcde"));
+    assertTrue(HtmlQuoting.needsQuoting("abc'de"));
+    assertTrue(HtmlQuoting.needsQuoting("abcde\""));
+    assertTrue(HtmlQuoting.needsQuoting("&"));
+    assertFalse(HtmlQuoting.needsQuoting(""));
+    assertFalse(HtmlQuoting.needsQuoting("ab\ncdef"));
+    assertFalse(HtmlQuoting.needsQuoting(null));
+  }
+
+  @Test public void testQuoting() throws Exception {
+    assertEquals("ab&lt;cd", HtmlQuoting.quoteHtmlChars("ab<cd"));
+    assertEquals("ab&gt;", HtmlQuoting.quoteHtmlChars("ab>"));
+    assertEquals("&amp;&amp;&amp;", HtmlQuoting.quoteHtmlChars("&&&"));
+    assertEquals(" &apos;\n", HtmlQuoting.quoteHtmlChars(" '\n"));
+    assertEquals("&quot;", HtmlQuoting.quoteHtmlChars("\""));
+    assertEquals(null, HtmlQuoting.quoteHtmlChars(null));
+  }
+
+  private void runRoundTrip(String str) throws Exception {
+    assertEquals(str, 
+                 HtmlQuoting.unquoteHtmlChars(HtmlQuoting.quoteHtmlChars(str)));
+  }
+  
+  @Test public void testRoundtrip() throws Exception {
+    runRoundTrip("");
+    runRoundTrip("<>&'\"");
+    runRoundTrip("ab>cd<ef&ghi'\"");
+    runRoundTrip("A string\n with no quotable chars in it!");
+    runRoundTrip(null);
+    StringBuilder buffer = new StringBuilder();
+    for(char ch=0; ch < 127; ++ch) {
+      buffer.append(ch);
+    }
+    runRoundTrip(buffer.toString());
+  }
+  
+
+  @Test
+  public void testRequestQuoting() throws Exception {
+    HttpServletRequest mockReq = Mockito.mock(HttpServletRequest.class);
+    HttpServer.QuotingInputFilter.RequestQuoter quoter =
+      new HttpServer.QuotingInputFilter.RequestQuoter(mockReq);
+    
+    Mockito.doReturn("a<b").when(mockReq).getParameter("x");
+    assertEquals("Test simple param quoting",
+        "a&lt;b", quoter.getParameter("x"));
+    
+    Mockito.doReturn(null).when(mockReq).getParameter("x");
+    assertEquals("Test that missing parameters dont cause NPE",
+        null, quoter.getParameter("x"));
+
+    Mockito.doReturn(new String[]{"a<b", "b"}).when(mockReq).getParameterValues("x");
+    assertArrayEquals("Test escaping of an array",
+        new String[]{"a&lt;b", "b"}, quoter.getParameterValues("x"));
+
+    Mockito.doReturn(null).when(mockReq).getParameterValues("x");
+    assertArrayEquals("Test that missing parameters dont cause NPE for array",
+        null, quoter.getParameterValues("x"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLog.java
new file mode 100644
index 0000000..db6cd1b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLog.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.HttpRequestLog;
+import org.apache.hadoop.hbase.http.HttpRequestLogAppender;
+import org.apache.log4j.Logger;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mortbay.jetty.NCSARequestLog;
+import org.mortbay.jetty.RequestLog;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+@Category(SmallTests.class)
+public class TestHttpRequestLog {
+
+  @Test
+  public void testAppenderUndefined() {
+    RequestLog requestLog = HttpRequestLog.getRequestLog("test");
+    assertNull("RequestLog should be null", requestLog);
+  }
+
+  @Test
+  public void testAppenderDefined() {
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setName("testrequestlog");
+    Logger.getLogger("http.requests.test").addAppender(requestLogAppender);
+    RequestLog requestLog = HttpRequestLog.getRequestLog("test");
+    Logger.getLogger("http.requests.test").removeAppender(requestLogAppender);
+    assertNotNull("RequestLog should not be null", requestLog);
+    assertEquals("Class mismatch", NCSARequestLog.class, requestLog.getClass());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLogAppender.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLogAppender.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLogAppender.java
new file mode 100644
index 0000000..208cbaa
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLogAppender.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.http;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.HttpRequestLogAppender;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+
+@Category(SmallTests.class)
+public class TestHttpRequestLogAppender {
+
+  @Test
+  public void testParameterPropagation() {
+
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setFilename("jetty-namenode-yyyy_mm_dd.log");
+    requestLogAppender.setRetainDays(17);
+    assertEquals("Filename mismatch", "jetty-namenode-yyyy_mm_dd.log",
+        requestLogAppender.getFilename());
+    assertEquals("Retain days mismatch", 17,
+        requestLogAppender.getRetainDays());
+  }
+}