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:41 UTC

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

Repository: hbase
Updated Branches:
  refs/heads/master 3d3784dd2 -> 26096f363


http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/resources/webapps/test/testjsp.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/webapps/test/testjsp.jsp b/hbase-server/src/test/resources/webapps/test/testjsp.jsp
new file mode 100644
index 0000000..23465da
--- /dev/null
+++ b/hbase-server/src/test/resources/webapps/test/testjsp.jsp
@@ -0,0 +1,21 @@
+<?xml version="1.0" encoding="UTF-8"?><%!
+/*
+ * 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.
+ */
+%>
+<%@ page contentType="text/html; charset=UTF-8" %> 
+Hello world!

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
index ee4acfb..97988eb 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
@@ -31,10 +31,9 @@ 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.security.User;
+import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
-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-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
index cbd636c..3208488 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
@@ -42,11 +42,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.filter.ParseFilter;
+import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.thrift.CallQueue;
 import org.apache.hadoop.hbase.thrift.CallQueue.Call;
 import org.apache.hadoop.hbase.thrift.ThriftMetrics;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
-import org.apache.hadoop.hbase.util.InfoServer;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TCompactProtocol;


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

Posted by st...@apache.org.
HBASE-10336 Remove deprecated usage of Hadoop HttpServer in InfoServer (Eric Charles)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/26096f36
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/26096f36
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/26096f36

Branch: refs/heads/master
Commit: 26096f363e8645ec61982ef26073e3172f1ce91c
Parents: 3d3784d
Author: Michael Stack <st...@apache.org>
Authored: Wed Jun 11 13:55:15 2014 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jun 11 13:55:15 2014 -0700

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        |   31 +
 hbase-server/pom.xml                            |    6 +
 .../hbase/http/AdminAuthorizedServlet.java      |   49 +
 .../hadoop/hbase/http/FilterContainer.java      |   41 +
 .../hadoop/hbase/http/FilterInitializer.java    |   32 +
 .../apache/hadoop/hbase/http/HtmlQuoting.java   |  215 +++
 .../apache/hadoop/hbase/http/HttpConfig.java    |   77 +
 .../hadoop/hbase/http/HttpRequestLog.java       |   92 ++
 .../hbase/http/HttpRequestLogAppender.java      |   63 +
 .../apache/hadoop/hbase/http/HttpServer.java    | 1362 ++++++++++++++++++
 .../apache/hadoop/hbase/http/InfoServer.java    |   90 ++
 .../apache/hadoop/hbase/http/NoCacheFilter.java |   52 +
 .../hbase/http/ServerConfigurationKeys.java     |   47 +
 .../hadoop/hbase/http/conf/ConfServlet.java     |  107 ++
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java   |  423 ++++++
 .../hadoop/hbase/http/jmx/package-info.java     |   26 +
 .../hbase/http/lib/StaticUserWebFilter.java     |  151 ++
 .../hadoop/hbase/http/lib/package-info.java     |   38 +
 .../apache/hadoop/hbase/http/log/LogLevel.java  |  175 +++
 .../apache/hadoop/hbase/http/package-info.java  |   28 +
 .../org/apache/hadoop/hbase/master/HMaster.java |    2 +
 .../hbase/regionserver/HRegionServer.java       |    2 +-
 .../apache/hadoop/hbase/rest/RESTServer.java    |    2 +-
 .../apache/hadoop/hbase/GenericTestUtils.java   |  365 +++++
 .../hadoop/hbase/TimedOutTestsListener.java     |  175 +++
 .../hbase/http/HttpServerFunctionalTest.java    |  232 +++
 .../hadoop/hbase/http/TestGlobalFilter.java     |  153 ++
 .../hadoop/hbase/http/TestHtmlQuoting.java      |   95 ++
 .../hadoop/hbase/http/TestHttpRequestLog.java   |   52 +
 .../hbase/http/TestHttpRequestLogAppender.java  |   41 +
 .../hadoop/hbase/http/TestHttpServer.java       |  613 ++++++++
 .../hbase/http/TestHttpServerLifecycle.java     |  153 ++
 .../hbase/http/TestHttpServerWebapps.java       |   68 +
 .../hadoop/hbase/http/TestPathFilter.java       |  157 ++
 .../hadoop/hbase/http/TestSSLHttpServer.java    |  121 ++
 .../hadoop/hbase/http/TestServletFilter.java    |  204 +++
 .../hadoop/hbase/http/conf/TestConfServlet.java |  116 ++
 .../hbase/http/jmx/TestJMXJsonServlet.java      |  108 ++
 .../hbase/http/lib/TestStaticUserWebFilter.java |   85 ++
 .../hadoop/hbase/http/log/TestLogLevel.java     |   85 ++
 .../hbase/http/resource/JerseyResource.java     |   64 +
 .../hadoop/hbase/http/ssl/KeyStoreTestUtil.java |  365 +++++
 .../src/test/resources/webapps/static/test.css  |   21 +
 .../src/test/resources/webapps/test/testjsp.jsp |   21 +
 .../hadoop/hbase/thrift/ThriftServer.java       |    3 +-
 .../hadoop/hbase/thrift2/ThriftServer.java      |    2 +-
 46 files changed, 6405 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 9773251..42ff032 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1201,4 +1201,35 @@ possible configurations would overwhelm and obscure the important.
     <value>org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager</value>
     <description>Fully qualified name of class implementing coordinated state manager.</description>
   </property>
+  <property>
+    <name>hbase.http.filter.initializers</name>
+    <value>org.apache.hadoop.hbase.http.lib.StaticUserWebFilter</value>
+    <description>
+      A comma separated list of class names. Each class in the list must extend 
+      org.apache.hadoop.hbase.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 default StaticUserWebFilter add a user principal as defined by the 
+      hbase.http.staticuser.user property.
+    </description>
+  </property>
+  <property>
+    <name>hbase.http.max.threads</name>
+    <value>10</value>
+    <description>
+      The maximum number of threads that the HTTP Server will create in its 
+      ThreadPool.
+    </description>
+  </property>
+  <!-- Static Web User Filter properties. -->
+  <property>
+    <description>
+      The user name to filter as, on static web filters
+      while rendering content. An example use is the HDFS
+      web UI (user to be used for browsing files).
+    </description>
+    <name>hbase.http.staticuser.user</name>
+    <value>dr.stack</value>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 0b31db1..69cce0c 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -29,6 +29,9 @@
   <artifactId>hbase-server</artifactId>
   <name>HBase - Server</name>
   <description>Main functionality for HBase</description>
+  <properties>
+    <test.build.webapps>target/test-classes/webapps</test.build.webapps>
+  </properties>
   <build>
     <!-- Makes sure the resources get added before they are processed
       by placing this first -->
@@ -210,6 +213,9 @@
               <value>org.apache.hadoop.hbase.ServerResourceCheckerJUnitListener</value>
             </property>
           </properties>
+          <systemPropertyVariables>
+            <test.build.webapps>target/test-classes/webapps</test.build.webapps>
+          </systemPropertyVariables>
         </configuration>
       </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/AdminAuthorizedServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/AdminAuthorizedServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/AdminAuthorizedServlet.java
new file mode 100644
index 0000000..0a7a028
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/AdminAuthorizedServlet.java
@@ -0,0 +1,49 @@
+/**
+ * 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.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.mortbay.jetty.servlet.DefaultServlet;
+
+/**
+ * General servlet which is admin-authorized.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class AdminAuthorizedServlet extends DefaultServlet {
+
+  private static final long serialVersionUID = 1L;
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+    throws ServletException, IOException {
+    // Do the authorization
+    if (HttpServer.hasAdministratorAccess(getServletContext(), request,
+        response)) {
+      // Authorization is done. Just call super.
+      super.doGet(request, response);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/FilterContainer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/FilterContainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/FilterContainer.java
new file mode 100644
index 0000000..7a79acc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/FilterContainer.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 java.util.Map;
+
+/**
+ * A container interface to add javax.servlet.Filter.
+ */
+public interface FilterContainer {
+  /**
+   * Add a filter to the container.
+   * @param name Filter name
+   * @param classname Filter class name
+   * @param parameters a map from parameter names to initial values
+   */
+  void addFilter(String name, String classname, Map<String, String> parameters);
+  /**
+   * Add a global filter to the container - This global filter will be
+   * applied to all available web contexts.
+   * @param name filter name
+   * @param classname filter class name
+   * @param parameters a map from parameter names to initial values
+   */
+  void addGlobalFilter(String name, String classname, Map<String, String> parameters);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/FilterInitializer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/FilterInitializer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/FilterInitializer.java
new file mode 100644
index 0000000..6b4223d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/FilterInitializer.java
@@ -0,0 +1,32 @@
+/**
+ * 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.conf.Configuration;
+
+/**
+ * Initialize a javax.servlet.Filter. 
+ */
+public abstract class FilterInitializer {
+  /**
+   * Initialize a Filter to a FilterContainer.
+   * @param container The filter container
+   * @param conf Configuration for run-time parameters
+   */
+  public abstract void initFilter(FilterContainer container, Configuration conf);
+}
\ 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/HtmlQuoting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java
new file mode 100644
index 0000000..60a74b7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java
@@ -0,0 +1,215 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * This class is responsible for quoting HTML characters.
+ */
+public class HtmlQuoting {
+  private static final byte[] ampBytes = "&amp;".getBytes();
+  private static final byte[] aposBytes = "&apos;".getBytes();
+  private static final byte[] gtBytes = "&gt;".getBytes();
+  private static final byte[] ltBytes = "&lt;".getBytes();
+  private static final byte[] quotBytes = "&quot;".getBytes();
+
+  /**
+   * Does the given string need to be quoted?
+   * @param data the string to check
+   * @param off the starting position
+   * @param len the number of bytes to check
+   * @return does the string contain any of the active html characters?
+   */
+  public static boolean needsQuoting(byte[] data, int off, int len) {
+    if (off+len > data.length) {
+        throw new IllegalStateException("off+len=" + off+len + " should be lower"
+                + " than data length=" + data.length);
+    }
+    for(int i=off; i< off+len; ++i) {
+      switch(data[i]) {
+      case '&':
+      case '<':
+      case '>':
+      case '\'':
+      case '"':
+        return true;
+      default:
+        break;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Does the given string need to be quoted?
+   * @param str the string to check
+   * @return does the string contain any of the active html characters?
+   */
+  public static boolean needsQuoting(String str) {
+    if (str == null) {
+      return false;
+    }
+    byte[] bytes = str.getBytes();
+    return needsQuoting(bytes, 0 , bytes.length);
+  }
+
+  /**
+   * Quote all of the active HTML characters in the given string as they
+   * are added to the buffer.
+   * @param output the stream to write the output to
+   * @param buffer the byte array to take the characters from
+   * @param off the index of the first byte to quote
+   * @param len the number of bytes to quote
+   */
+  public static void quoteHtmlChars(OutputStream output, byte[] buffer,
+                                    int off, int len) throws IOException {
+    for(int i=off; i < off+len; i++) {
+      switch (buffer[i]) {
+      case '&': output.write(ampBytes); break;
+      case '<': output.write(ltBytes); break;
+      case '>': output.write(gtBytes); break;
+      case '\'': output.write(aposBytes); break;
+      case '"': output.write(quotBytes); break;
+      default: output.write(buffer, i, 1);
+      }
+    }
+  }
+  
+  /**
+   * Quote the given item to make it html-safe.
+   * @param item the string to quote
+   * @return the quoted string
+   */
+  public static String quoteHtmlChars(String item) {
+    if (item == null) {
+      return null;
+    }
+    byte[] bytes = item.getBytes();
+    if (needsQuoting(bytes, 0, bytes.length)) {
+      ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+      try {
+        quoteHtmlChars(buffer, bytes, 0, bytes.length);
+      } catch (IOException ioe) {
+        // Won't happen, since it is a bytearrayoutputstream
+      }
+      return buffer.toString();
+    } else {
+      return item;
+    }
+  }
+
+  /**
+   * Return an output stream that quotes all of the output.
+   * @param out the stream to write the quoted output to
+   * @return a new stream that the application show write to
+   * @throws IOException if the underlying output fails
+   */
+  public static OutputStream quoteOutputStream(final OutputStream out
+                                               ) throws IOException {
+    return new OutputStream() {
+      private byte[] data = new byte[1];
+      @Override
+      public void write(byte[] data, int off, int len) throws IOException {
+        quoteHtmlChars(out, data, off, len);
+      }
+      
+      @Override
+      public void write(int b) throws IOException {
+        data[0] = (byte) b;
+        quoteHtmlChars(out, data, 0, 1);
+      }
+      
+      @Override
+      public void flush() throws IOException {
+        out.flush();
+      }
+      
+      @Override
+      public void close() throws IOException {
+        out.close();
+      }
+    };
+  }
+
+  /**
+   * Remove HTML quoting from a string.
+   * @param item the string to unquote
+   * @return the unquoted string
+   */
+  public static String unquoteHtmlChars(String item) {
+    if (item == null) {
+      return null;
+    }
+    int next = item.indexOf('&');
+    // nothing was quoted
+    if (next == -1) {
+      return item;
+    }
+    int len = item.length();
+    int posn = 0;
+    StringBuilder buffer = new StringBuilder();
+    while (next != -1) {
+      buffer.append(item.substring(posn, next));
+      if (item.startsWith("&amp;", next)) {
+        buffer.append('&');
+        next += 5;
+      } else if (item.startsWith("&apos;", next)) {
+        buffer.append('\'');
+        next += 6;        
+      } else if (item.startsWith("&gt;", next)) {
+        buffer.append('>');
+        next += 4;
+      } else if (item.startsWith("&lt;", next)) {
+        buffer.append('<');
+        next += 4;
+      } else if (item.startsWith("&quot;", next)) {
+        buffer.append('"');
+        next += 6;
+      } else {
+        int end = item.indexOf(';', next)+1;
+        if (end == 0) {
+          end = len;
+        }
+        throw new IllegalArgumentException("Bad HTML quoting for " + 
+                                           item.substring(next,end));
+      }
+      posn = next;
+      next = item.indexOf('&', posn);
+    }
+    buffer.append(item.substring(posn, len));
+    return buffer.toString();
+  }
+  
+  public static void main(String[] args) throws Exception {
+    if (args.length == 0) {
+        throw new IllegalArgumentException("Please provide some arguments");
+    }
+    for(String arg:args) {
+      System.out.println("Original: " + arg);
+      String quoted = quoteHtmlChars(arg);
+      System.out.println("Quoted: "+ quoted);
+      String unquoted = unquoteHtmlChars(quoted);
+      System.out.println("Unquoted: " + unquoted);
+      System.out.println();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
new file mode 100644
index 0000000..c040aa7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
@@ -0,0 +1,77 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Statics to get access to Http related configuration.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HttpConfig {
+  private static Policy policy;
+  public enum Policy {
+    HTTP_ONLY,
+    HTTPS_ONLY,
+    HTTP_AND_HTTPS;
+
+    public static Policy fromString(String value) {
+      if (HTTPS_ONLY.name().equalsIgnoreCase(value)) {
+        return HTTPS_ONLY;
+      } else if (HTTP_AND_HTTPS.name().equalsIgnoreCase(value)) {
+        return HTTP_AND_HTTPS;
+      }
+      return HTTP_ONLY;
+    }
+
+    public boolean isHttpEnabled() {
+      return this == HTTP_ONLY || this == HTTP_AND_HTTPS;
+    }
+
+    public boolean isHttpsEnabled() {
+      return this == HTTPS_ONLY || this == HTTP_AND_HTTPS;
+    }
+  }
+
+  static {
+    Configuration conf = new Configuration();
+    boolean sslEnabled = conf.getBoolean(
+            ServerConfigurationKeys.HBASE_SSL_ENABLED_KEY,
+            ServerConfigurationKeys.HBASE_SSL_ENABLED_DEFAULT);
+    policy = sslEnabled ? Policy.HTTPS_ONLY : Policy.HTTP_ONLY;
+  }
+
+  public static void setPolicy(Policy policy) {
+    HttpConfig.policy = policy;
+  }
+
+  public static boolean isSecure() {
+    return policy == Policy.HTTPS_ONLY;
+  }
+
+  public static String getSchemePrefix() {
+    return (isSecure()) ? "https://" : "http://";
+  }
+
+  public static String getScheme(Policy policy) {
+    return policy == Policy.HTTPS_ONLY ? "https://" : "http://";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java
new file mode 100644
index 0000000..def36a0
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java
@@ -0,0 +1,92 @@
+/**
+ * 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.util.HashMap;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogConfigurationException;
+import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Appender;
+import org.apache.log4j.Logger;
+import org.mortbay.jetty.NCSARequestLog;
+import org.mortbay.jetty.RequestLog;
+
+/**
+ * RequestLog object for use with Http
+ */
+public class HttpRequestLog {
+
+  public static final Log LOG = LogFactory.getLog(HttpRequestLog.class);
+  private static final HashMap<String, String> serverToComponent;
+
+  static {
+    serverToComponent = new HashMap<String, String>();
+    serverToComponent.put("master", "master");
+    serverToComponent.put("region", "regionserver");
+  }
+
+  public static RequestLog getRequestLog(String name) {
+
+    String lookup = serverToComponent.get(name);
+    if (lookup != null) {
+      name = lookup;
+    }
+    String loggerName = "http.requests." + name;
+    String appenderName = name + "requestlog";
+    Log logger = LogFactory.getLog(loggerName);
+
+    if (logger instanceof Log4JLogger) {
+      Log4JLogger httpLog4JLog = (Log4JLogger)logger;
+      Logger httpLogger = httpLog4JLog.getLogger();
+      Appender appender = null;
+
+      try {
+        appender = httpLogger.getAppender(appenderName);
+      } catch (LogConfigurationException e) {
+        LOG.warn("Http request log for " + loggerName
+            + " could not be created");
+        throw e;
+      }
+
+      if (appender == null) {
+        LOG.info("Http request log for " + loggerName
+            + " is not defined");
+        return null;
+      }
+
+      if (appender instanceof HttpRequestLogAppender) {
+        HttpRequestLogAppender requestLogAppender
+          = (HttpRequestLogAppender)appender;
+        NCSARequestLog requestLog = new NCSARequestLog();
+        requestLog.setFilename(requestLogAppender.getFilename());
+        requestLog.setRetainDays(requestLogAppender.getRetainDays());
+        return requestLog;
+      } else {
+        LOG.warn("Jetty request log for " + loggerName
+            + " was of the wrong class");
+        return null;
+      }
+    }
+    else {
+      LOG.warn("Jetty request log can only be enabled using Log4j");
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLogAppender.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLogAppender.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLogAppender.java
new file mode 100644
index 0000000..8039b34
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLogAppender.java
@@ -0,0 +1,63 @@
+/**
+ * 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.log4j.spi.LoggingEvent;
+import org.apache.log4j.AppenderSkeleton;
+
+/**
+ * Log4j Appender adapter for HttpRequestLog
+ */
+public class HttpRequestLogAppender extends AppenderSkeleton {
+
+  private String filename;
+  private int retainDays;
+
+  public HttpRequestLogAppender() {
+  }
+
+  public void setRetainDays(int retainDays) {
+    this.retainDays = retainDays;
+  }
+
+  public int getRetainDays() {
+    return retainDays;
+  }
+
+  public void setFilename(String filename) {
+    this.filename = filename;
+  }
+
+  public String getFilename() {
+    return filename;
+  }
+
+  @Override
+  public void append(LoggingEvent event) {
+  }
+
+  @Override
+  public void close() {
+      // Do nothing, we don't have close() on AppenderSkeleton.
+  }
+
+  @Override
+  public boolean requiresLayout() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
new file mode 100644
index 0000000..857d0ad
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
@@ -0,0 +1,1362 @@
+/**
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.io.PrintWriter;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.http.conf.ConfServlet;
+import org.apache.hadoop.hbase.http.jmx.JMXJsonServlet;
+import org.apache.hadoop.hbase.http.log.LogLevel;
+import org.apache.hadoop.metrics.MetricsServlet;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Shell;
+import org.mortbay.io.Buffer;
+import org.mortbay.jetty.Connector;
+import org.mortbay.jetty.Handler;
+import org.mortbay.jetty.MimeTypes;
+import org.mortbay.jetty.RequestLog;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.handler.ContextHandler;
+import org.mortbay.jetty.handler.ContextHandlerCollection;
+import org.mortbay.jetty.handler.HandlerCollection;
+import org.mortbay.jetty.handler.RequestLogHandler;
+import org.mortbay.jetty.nio.SelectChannelConnector;
+import org.mortbay.jetty.security.SslSocketConnector;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.DefaultServlet;
+import org.mortbay.jetty.servlet.FilterHolder;
+import org.mortbay.jetty.servlet.FilterMapping;
+import org.mortbay.jetty.servlet.ServletHandler;
+import org.mortbay.jetty.servlet.ServletHolder;
+import org.mortbay.jetty.webapp.WebAppContext;
+import org.mortbay.thread.QueuedThreadPool;
+import org.mortbay.util.MultiException;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.sun.jersey.spi.container.servlet.ServletContainer;
+
+/**
+ * Create a Jetty embedded server to answer http requests. The primary goal
+ * is to serve up status information for the server.
+ * There are three contexts:
+ *   "/logs/" -> points to the log directory
+ *   "/static/" -> points to common static files (src/webapps/static)
+ *   "/" -> the jsp server code from (src/webapps/<name>)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HttpServer implements FilterContainer {
+  public static final Log LOG = LogFactory.getLog(HttpServer.class);
+
+  static final String FILTER_INITIALIZERS_PROPERTY
+      = "hbase.http.filter.initializers";
+  static final String HTTP_MAX_THREADS = "hbase.http.max.threads";
+
+  // The ServletContext attribute where the daemon Configuration
+  // gets stored.
+  public static final String CONF_CONTEXT_ATTRIBUTE = "hbase.conf";
+  public static final String ADMINS_ACL = "admins.acl";
+  public static final String BIND_ADDRESS = "bind.address";
+  public static final String SPNEGO_FILTER = "SpnegoFilter";
+  public static final String NO_CACHE_FILTER = "NoCacheFilter";
+  public static final String APP_DIR = "webapps";
+
+  private final AccessControlList adminsAcl;
+
+  protected final Server webServer;
+  protected String appDir;
+  protected String logDir;
+
+  private static class ListenerInfo {
+    /**
+     * Boolean flag to determine whether the HTTP server should clean up the
+     * listener in stop().
+     */
+    private final boolean isManaged;
+    private final Connector listener;
+    private ListenerInfo(boolean isManaged, Connector listener) {
+      this.isManaged = isManaged;
+      this.listener = listener;
+    }
+  }
+
+  private final List<ListenerInfo> listeners = Lists.newArrayList();
+
+  protected final WebAppContext webAppContext;
+  protected final boolean findPort;
+  protected final Map<Context, Boolean> defaultContexts =
+      new HashMap<Context, Boolean>();
+  protected final List<String> filterNames = new ArrayList<String>();
+  static final String STATE_DESCRIPTION_ALIVE = " - alive";
+  static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
+
+  /**
+   * Class to construct instances of HTTP server with specific options.
+   */
+  public static class Builder {
+    private ArrayList<URI> endpoints = Lists.newArrayList();
+    private Connector connector;
+    private Configuration conf;
+    private String[] pathSpecs;
+    private AccessControlList adminsAcl;
+    private boolean securityEnabled = false;
+    private String usernameConfKey;
+    private String keytabConfKey;
+    private boolean needsClientAuth;
+
+    private String hostName;
+    private String appDir = APP_DIR;
+    private String logDir;
+    private boolean findPort;
+
+    private String trustStore;
+    private String trustStorePassword;
+    private String trustStoreType;
+
+    private String keyStore;
+    private String keyStorePassword;
+    private String keyStoreType;
+
+    // The -keypass option in keytool
+    private String keyPassword;
+
+    @Deprecated
+    private String name;
+    @Deprecated
+    private String bindAddress;
+    @Deprecated
+    private int port = -1;
+
+    /**
+     * Add an endpoint that the HTTP server should listen to.
+     *
+     * @param endpoint
+     *          the endpoint of that the HTTP server should listen to. The
+     *          scheme specifies the protocol (i.e. HTTP / HTTPS), the host
+     *          specifies the binding address, and the port specifies the
+     *          listening port. Unspecified or zero port means that the server
+     *          can listen to any port.
+     */
+    public Builder addEndpoint(URI endpoint) {
+      endpoints.add(endpoint);
+      return this;
+    }
+
+    /**
+     * Set the hostname of the http server. The host name is used to resolve the
+     * _HOST field in Kerberos principals. The hostname of the first listener
+     * will be used if the name is unspecified.
+     */
+    public Builder hostName(String hostName) {
+      this.hostName = hostName;
+      return this;
+    }
+    
+    public Builder trustStore(String location, String password, String type) {
+      this.trustStore = location;
+      this.trustStorePassword = password;
+      this.trustStoreType = type;
+      return this;
+    }
+
+    public Builder keyStore(String location, String password, String type) {
+      this.keyStore = location;
+      this.keyStorePassword = password;
+      this.keyStoreType = type;
+      return this;
+    }
+
+    public Builder keyPassword(String password) {
+      this.keyPassword = password;
+      return this;
+    }
+
+    /**
+     * Specify whether the server should authorize the client in SSL
+     * connections.
+     */
+    public Builder needsClientAuth(boolean value) {
+      this.needsClientAuth = value;
+      return this;
+    }
+
+    /**
+     * Use setAppDir() instead.
+     */
+    @Deprecated
+    public Builder setName(String name){
+      this.name = name;
+      return this;
+    }
+
+    /**
+     * Use addEndpoint() instead.
+     */
+    @Deprecated
+    public Builder setBindAddress(String bindAddress){
+      this.bindAddress = bindAddress;
+      return this;
+    }
+
+    /**
+     * Use addEndpoint() instead.
+     */
+    @Deprecated
+    public Builder setPort(int port) {
+      this.port = port;
+      return this;
+    }
+    
+    public Builder setFindPort(boolean findPort) {
+      this.findPort = findPort;
+      return this;
+    }
+    
+    public Builder setConf(Configuration conf) {
+      this.conf = conf;
+      return this;
+    }
+    
+    public Builder setConnector(Connector connector) {
+      this.connector = connector;
+      return this;
+    }
+    
+    public Builder setPathSpec(String[] pathSpec) {
+      this.pathSpecs = pathSpec;
+      return this;
+    }
+    
+    public Builder setACL(AccessControlList acl) {
+      this.adminsAcl = acl;
+      return this;
+    }
+    
+    public Builder setSecurityEnabled(boolean securityEnabled) {
+      this.securityEnabled = securityEnabled;
+      return this;
+    }
+    
+    public Builder setUsernameConfKey(String usernameConfKey) {
+      this.usernameConfKey = usernameConfKey;
+      return this;
+    }
+
+    public Builder setKeytabConfKey(String keytabConfKey) {
+      this.keytabConfKey = keytabConfKey;
+      return this;
+    }
+
+    public Builder setAppDir(String appDir) {
+        this.appDir = appDir;
+        return this;
+      }
+
+    public Builder setLogDir(String logDir) {
+        this.logDir = logDir;
+        return this;
+      }
+
+    public HttpServer build() throws IOException {
+
+      // Do we still need to assert this non null name if it is deprecated?
+      if (this.name == null) {
+        throw new HadoopIllegalArgumentException("name is not set");
+      }
+
+      // Make the behavior compatible with deprecated interfaces
+      if (bindAddress != null && port != -1) {
+        try {
+          endpoints.add(0, new URI("http", "", bindAddress, port, "", "", ""));
+        } catch (URISyntaxException e) {
+          throw new HadoopIllegalArgumentException("Invalid endpoint: "+ e);
+        }
+      }
+
+      if (endpoints.size() == 0 && connector == null) {
+        throw new HadoopIllegalArgumentException("No endpoints specified");
+      }
+
+      if (hostName == null) {
+        hostName = endpoints.size() == 0 ? connector.getHost() : endpoints.get(
+            0).getHost();
+      }
+      
+      if (this.conf == null) {
+        conf = new Configuration();
+      }
+
+      HttpServer server = new HttpServer(this);
+
+      if (this.securityEnabled) {
+        server.initSpnego(conf, hostName, usernameConfKey, keytabConfKey);
+      }
+
+      if (connector != null) {
+        server.addUnmanagedListener(connector);
+      }
+
+      for (URI ep : endpoints) {
+        Connector listener = null;
+        String scheme = ep.getScheme();
+        if ("http".equals(scheme)) {
+          listener = HttpServer.createDefaultChannelConnector();
+        } else if ("https".equals(scheme)) {
+          SslSocketConnector c = new SslSocketConnector();
+          c.setNeedClientAuth(needsClientAuth);
+          c.setKeyPassword(keyPassword);
+
+          if (keyStore != null) {
+            c.setKeystore(keyStore);
+            c.setKeystoreType(keyStoreType);
+            c.setPassword(keyStorePassword);
+          }
+
+          if (trustStore != null) {
+            c.setTruststore(trustStore);
+            c.setTruststoreType(trustStoreType);
+            c.setTrustPassword(trustStorePassword);
+          }
+          listener = c;
+
+        } else {
+          throw new HadoopIllegalArgumentException(
+              "unknown scheme for endpoint:" + ep);
+        }
+        listener.setHost(ep.getHost());
+        listener.setPort(ep.getPort() == -1 ? 0 : ep.getPort());
+        server.addManagedListener(listener);
+      }
+
+      server.loadListeners();
+      return server;
+
+    }
+
+  }
+  
+  /** Same as this(name, bindAddress, port, findPort, null); */
+  @Deprecated
+  public HttpServer(String name, String bindAddress, int port, boolean findPort
+      ) throws IOException {
+    this(name, bindAddress, port, findPort, new Configuration());
+  }
+
+  @Deprecated
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf, Connector connector) throws IOException {
+    this(name, bindAddress, port, findPort, conf, null, connector, null);
+  }
+
+  /**
+   * Create a status server on the given port. Allows you to specify the
+   * path specifications that this server will be serving so that they will be
+   * added to the filters properly.  
+   * 
+   * @param name The name of the server
+   * @param bindAddress The address for this server
+   * @param port The port to use on the server
+   * @param findPort whether the server should start at the given port and 
+   *        increment by 1 until it finds a free port.
+   * @param conf Configuration 
+   * @param pathSpecs Path specifications that this httpserver will be serving. 
+   *        These will be added to any filters.
+   */
+  @Deprecated
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
+    this(name, bindAddress, port, findPort, conf, null, null, pathSpecs);
+  }
+  
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/webapps/<name>.
+   * @param name The name of the server
+   * @param port The port to use on the server
+   * @param findPort whether the server should start at the given port and 
+   *        increment by 1 until it finds a free port.
+   * @param conf Configuration 
+   */
+  @Deprecated
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf) throws IOException {
+    this(name, bindAddress, port, findPort, conf, null, null, null);
+  }
+
+  @Deprecated
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf, AccessControlList adminsAcl) 
+      throws IOException {
+    this(name, bindAddress, port, findPort, conf, adminsAcl, null, null);
+  }
+
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/webapps/<name>.
+   * @param name The name of the server
+   * @param bindAddress The address for this server
+   * @param port The port to use on the server
+   * @param findPort whether the server should start at the given port and 
+   *        increment by 1 until it finds a free port.
+   * @param conf Configuration 
+   * @param adminsAcl {@link AccessControlList} of the admins
+   * @param connector The jetty {@link Connector} to use
+   */
+  @Deprecated
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf, AccessControlList adminsAcl, 
+      Connector connector) throws IOException {
+    this(name, bindAddress, port, findPort, conf, adminsAcl, connector, null);
+  }
+
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/webapps/<name>.
+   * @param name The name of the server
+   * @param bindAddress The address for this server
+   * @param port The port to use on the server
+   * @param findPort whether the server should start at the given port and 
+   *        increment by 1 until it finds a free port.
+   * @param conf Configuration 
+   * @param adminsAcl {@link AccessControlList} of the admins
+   * @param connector A jetty connection listener
+   * @param pathSpecs Path specifications that this httpserver will be serving. 
+   *        These will be added to any filters.
+   */
+  @Deprecated
+  public HttpServer(String name, String bindAddress, int port,
+      boolean findPort, Configuration conf, AccessControlList adminsAcl, 
+      Connector connector, String[] pathSpecs) throws IOException {
+    this(new Builder().setName(name)
+        .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
+        .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
+        .setConnector(connector).setPathSpec(pathSpecs));
+  }
+
+  private HttpServer(final Builder b) throws IOException {
+    this.appDir = b.appDir;
+    this.logDir = b.logDir;
+    final String appDir = getWebAppsPath(b.name);
+    this.webServer = new Server();
+    this.adminsAcl = b.adminsAcl;
+    this.webAppContext = createWebAppContext(b.name, b.conf, adminsAcl, appDir);
+    this.findPort = b.findPort;
+    initializeWebServer(b.name, b.hostName, b.conf, b.pathSpecs);
+  }
+
+  private void initializeWebServer(String name, String hostName,
+      Configuration conf, String[] pathSpecs)
+      throws FileNotFoundException, IOException {
+
+    Preconditions.checkNotNull(webAppContext);
+
+    int maxThreads = conf.getInt(HTTP_MAX_THREADS, -1);
+    // If HTTP_MAX_THREADS is not configured, QueueThreadPool() will use the
+    // default value (currently 250).
+    QueuedThreadPool threadPool = maxThreads == -1 ? new QueuedThreadPool()
+        : new QueuedThreadPool(maxThreads);
+    threadPool.setDaemon(true);
+    webServer.setThreadPool(threadPool);
+
+    ContextHandlerCollection contexts = new ContextHandlerCollection();
+    RequestLog requestLog = HttpRequestLog.getRequestLog(name);
+
+    if (requestLog != null) {
+      RequestLogHandler requestLogHandler = new RequestLogHandler();
+      requestLogHandler.setRequestLog(requestLog);
+      HandlerCollection handlers = new HandlerCollection();
+      handlers.setHandlers(new Handler[] { requestLogHandler, contexts });
+      webServer.setHandler(handlers);
+    } else {
+      webServer.setHandler(contexts);
+    }
+
+    final String appDir = getWebAppsPath(name);
+
+    webServer.addHandler(webAppContext);
+
+    addDefaultApps(contexts, appDir, conf);
+
+    addGlobalFilter("safety", QuotingInputFilter.class.getName(), null);
+    final FilterInitializer[] initializers = getFilterInitializers(conf);
+    if (initializers != null) {
+      conf = new Configuration(conf);
+      conf.set(BIND_ADDRESS, hostName);
+      for (FilterInitializer c : initializers) {
+        c.initFilter(this, conf);
+      }
+    }
+
+    addDefaultServlets();
+
+    if (pathSpecs != null) {
+      for (String path : pathSpecs) {
+        LOG.info("adding path spec: " + path);
+        addFilterPathMapping(path, webAppContext);
+      }
+    }
+  }
+
+  private void addUnmanagedListener(Connector connector) {
+    listeners.add(new ListenerInfo(false, connector));
+  }
+
+  private void addManagedListener(Connector connector) {
+    listeners.add(new ListenerInfo(true, connector));
+  }
+
+  private static WebAppContext createWebAppContext(String name,
+      Configuration conf, AccessControlList adminsAcl, final String appDir) {
+    WebAppContext ctx = new WebAppContext();
+    ctx.setDisplayName(name);
+    ctx.setContextPath("/");
+    ctx.setWar(appDir + "/" + name);
+    ctx.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
+    ctx.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
+    addNoCacheFilter(ctx);
+    return ctx;
+  }
+
+  private static void addNoCacheFilter(WebAppContext ctxt) {
+    defineFilter(ctxt, NO_CACHE_FILTER, NoCacheFilter.class.getName(),
+        Collections.<String, String> emptyMap(), new String[] { "/*" });
+  }
+
+  /**
+   * Create a required listener for the Jetty instance listening on the port
+   * provided. This wrapper and all subclasses must create at least one
+   * listener.
+   */
+  public Connector createBaseListener(Configuration conf) throws IOException {
+    return HttpServer.createDefaultChannelConnector();
+  }
+  
+  @InterfaceAudience.Private
+  public static Connector createDefaultChannelConnector() {
+    SelectChannelConnector ret = new SelectChannelConnector();
+    ret.setLowResourceMaxIdleTime(10000);
+    ret.setAcceptQueueSize(128);
+    ret.setResolveNames(false);
+    ret.setUseDirectBuffers(false);
+    if(Shell.WINDOWS) {
+      // result of setting the SO_REUSEADDR flag is different on Windows
+      // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx
+      // without this 2 NN's can start on the same machine and listen on 
+      // the same port with indeterminate routing of incoming requests to them
+      ret.setReuseAddress(false);
+    }
+    ret.setHeaderBufferSize(1024*64);
+    return ret;
+  }
+
+  /** 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_INITIALIZERS_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 default apps.
+   * @param appDir The application directory
+   * @throws IOException
+   */
+  protected void addDefaultApps(ContextHandlerCollection parent,
+      final String appDir, Configuration conf) throws IOException {
+    // set up the context for "/logs/" if "hadoop.log.dir" property is defined. 
+    String logDir = this.logDir;
+    if (logDir == null) {
+        logDir = System.getProperty("hadoop.log.dir");
+    }
+    if (logDir != null) {
+      Context logContext = new Context(parent, "/logs");
+      logContext.setResourceBase(logDir);
+      logContext.addServlet(AdminAuthorizedServlet.class, "/*");
+      if (conf.getBoolean(
+          ServerConfigurationKeys.HBASE_JETTY_LOGS_SERVE_ALIASES,
+          ServerConfigurationKeys.DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES)) {
+        @SuppressWarnings("unchecked")
+        Map<String, String> params = logContext.getInitParams();
+        params.put(
+            "org.mortbay.jetty.servlet.Default.aliases", "true");
+      }
+      logContext.setDisplayName("logs");
+      setContextAttributes(logContext, conf);
+      addNoCacheFilter(webAppContext);
+      defaultContexts.put(logContext, true);
+    }
+    // set up the context for "/static/*"
+    Context staticContext = new Context(parent, "/static");
+    staticContext.setResourceBase(appDir + "/static");
+    staticContext.addServlet(DefaultServlet.class, "/*");
+    staticContext.setDisplayName("static");
+    setContextAttributes(staticContext, conf);
+    defaultContexts.put(staticContext, true);
+  }
+  
+  private void setContextAttributes(Context context, Configuration conf) {
+    context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
+    context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
+  }
+
+  /**
+   * Add default servlets.
+   */
+  protected void addDefaultServlets() {
+    // set up default servlets
+    addServlet("stacks", "/stacks", StackServlet.class);
+    addServlet("logLevel", "/logLevel", LogLevel.Servlet.class);
+    addServlet("metrics", "/metrics", MetricsServlet.class);
+    addServlet("jmx", "/jmx", JMXJsonServlet.class);
+    addServlet("conf", "/conf", ConfServlet.class);
+  }
+
+  public void addContext(Context ctxt, boolean isFiltered)
+      throws IOException {
+    webServer.addHandler(ctxt);
+    addNoCacheFilter(webAppContext);
+    defaultContexts.put(ctxt, isFiltered);
+  }
+
+  /**
+   * 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 
+   * @throws IOException
+   */
+  protected void addContext(String pathSpec, String dir, boolean isFiltered) throws IOException {
+    if (0 == webServer.getHandlers().length) {
+      throw new RuntimeException("Couldn't find handler");
+    }
+    WebAppContext webAppCtx = new WebAppContext();
+    webAppCtx.setContextPath(pathSpec);
+    webAppCtx.setWar(dir);
+    addContext(webAppCtx, true);
+  }
+
+  /**
+   * Set a value in the webapp context. These values are available to the jsp
+   * pages as "application.getAttribute(name)".
+   * @param name The name of the attribute
+   * @param value The value of the attribute
+   */
+  public void setAttribute(String name, Object value) {
+    webAppContext.setAttribute(name, value);
+  }
+
+  /** 
+   * Add a Jersey resource package.
+   * @param packageName The Java package name containing the Jersey resource.
+   * @param pathSpec The path spec for the servlet
+   */
+  public void addJerseyResourcePackage(final String packageName,
+      final String pathSpec) {
+    LOG.info("addJerseyResourcePackage: packageName=" + packageName
+        + ", pathSpec=" + pathSpec);
+    final ServletHolder sh = new ServletHolder(ServletContainer.class);
+    sh.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
+        "com.sun.jersey.api.core.PackagesResourceConfig");
+    sh.setInitParameter("com.sun.jersey.config.property.packages", packageName);
+    webAppContext.addServlet(sh, pathSpec);
+  }
+
+  /**
+   * Add a 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
+   */
+  public void addServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz) {
+    addInternalServlet(name, pathSpec, clazz, false);
+    addFilterPathMapping(pathSpec, webAppContext);
+  }
+
+  /**
+   * Add an internal servlet in the server. 
+   * Note: This method is to be used for adding servlets that facilitate
+   * internal communication and not for user facing functionality. For
+   * servlets added using this method, filters are not enabled. 
+   * 
+   * @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
+   */
+  public void addInternalServlet(String name, String pathSpec,
+      Class<? extends HttpServlet> clazz) {
+    addInternalServlet(name, pathSpec, clazz, false);
+  }
+
+  /**
+   * Add an internal servlet in the server, specifying whether or not to
+   * protect with Kerberos authentication. 
+   * Note: This method is to be used for adding servlets that facilitate
+   * internal communication and not for user facing functionality. For
+   +   * servlets added using this method, filters (except internal Kerberos
+   * filters) are not enabled. 
+   * 
+   * @param name The name of the servlet (can be passed as null)
+   * @param pathSpec The path spec for the servlet
+   * @param clazz The servlet class
+   * @param requireAuth Require Kerberos authenticate to access servlet
+   */
+  public void addInternalServlet(String name, String pathSpec, 
+      Class<? extends HttpServlet> clazz, boolean requireAuth) {
+    ServletHolder holder = new ServletHolder(clazz);
+    if (name != null) {
+      holder.setName(name);
+    }
+    webAppContext.addServlet(holder, pathSpec);
+
+    if(requireAuth && UserGroupInformation.isSecurityEnabled()) {
+       LOG.info("Adding Kerberos (SPNEGO) filter to " + name);
+       ServletHandler handler = webAppContext.getServletHandler();
+       FilterMapping fmap = new FilterMapping();
+       fmap.setPathSpec(pathSpec);
+       fmap.setFilterName(SPNEGO_FILTER);
+       fmap.setDispatches(Handler.ALL);
+       handler.addFilterMapping(fmap);
+    }
+  }
+
+  @Override
+  public void addFilter(String name, String classname,
+      Map<String, String> parameters) {
+
+    final String[] USER_FACING_URLS = { "*.html", "*.jsp" };
+    defineFilter(webAppContext, name, classname, parameters, USER_FACING_URLS);
+    LOG.info("Added filter " + name + " (class=" + classname
+        + ") to context " + webAppContext.getDisplayName());
+    final String[] ALL_URLS = { "/*" };
+    for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
+      if (e.getValue()) {
+        Context ctx = e.getKey();
+        defineFilter(ctx, name, classname, parameters, ALL_URLS);
+        LOG.info("Added filter " + name + " (class=" + classname
+            + ") to context " + ctx.getDisplayName());
+      }
+    }
+    filterNames.add(name);
+  }
+
+  @Override
+  public void addGlobalFilter(String name, String classname,
+      Map<String, String> parameters) {
+    final String[] ALL_URLS = { "/*" };
+    defineFilter(webAppContext, name, classname, parameters, ALL_URLS);
+    for (Context ctx : defaultContexts.keySet()) {
+      defineFilter(ctx, name, classname, parameters, ALL_URLS);
+    }
+    LOG.info("Added global filter '" + name + "' (class=" + classname + ")");
+  }
+
+  /**
+   * Define a filter for a context and set up default url mappings.
+   */
+  public static void defineFilter(Context ctx, String name,
+      String classname, Map<String,String> parameters, String[] urls) {
+
+    FilterHolder holder = new FilterHolder();
+    holder.setName(name);
+    holder.setClassName(classname);
+    holder.setInitParameters(parameters);
+    FilterMapping fmap = new FilterMapping();
+    fmap.setPathSpecs(urls);
+    fmap.setDispatches(Handler.ALL);
+    fmap.setFilterName(name);
+    ServletHandler handler = ctx.getServletHandler();
+    handler.addFilter(holder, fmap);
+  }
+
+  /**
+   * Add the path spec to the filter path mapping.
+   * @param pathSpec The path spec
+   * @param webAppCtx The WebApplicationContext to add to
+   */
+  protected void addFilterPathMapping(String pathSpec,
+      Context webAppCtx) {
+    ServletHandler handler = webAppCtx.getServletHandler();
+    for(String name : filterNames) {
+      FilterMapping fmap = new FilterMapping();
+      fmap.setPathSpec(pathSpec);
+      fmap.setFilterName(name);
+      fmap.setDispatches(Handler.ALL);
+      handler.addFilterMapping(fmap);
+    }
+  }
+  
+  /**
+   * Get the value in the webapp context.
+   * @param name The name of the attribute
+   * @return The value of the attribute
+   */
+  public Object getAttribute(String name) {
+    return webAppContext.getAttribute(name);
+  }
+  
+  public WebAppContext getWebAppContext(){
+    return this.webAppContext;
+  }
+
+  public String getWebAppsPath(String appName) throws FileNotFoundException {
+      return getWebAppsPath(this.appDir, appName);
+  }
+
+  /**
+   * Get the pathname to the webapps files.
+   * @param appName eg "secondary" or "datanode"
+   * @return the pathname as a URL
+   * @throws FileNotFoundException if 'webapps' directory cannot be found on CLASSPATH.
+   */
+  protected String getWebAppsPath(String webapps, String appName) throws FileNotFoundException {
+    URL url = getClass().getClassLoader().getResource(webapps + "/" + appName);
+    if (url == null) 
+      throw new FileNotFoundException(webapps + "/" + appName
+          + " not found in CLASSPATH");
+    String urlString = url.toString();
+    return urlString.substring(0, urlString.lastIndexOf('/'));
+  }
+
+  /**
+   * Get the port that the server is on
+   * @return the port
+   */
+  @Deprecated
+  public int getPort() {
+    return webServer.getConnectors()[0].getLocalPort();
+  }
+
+  /**
+   * Get the address that corresponds to a particular connector.
+   *
+   * @return the corresponding address for the connector, or null if there's no
+   *         such connector or the connector is not bounded.
+   */
+  public InetSocketAddress getConnectorAddress(int index) {
+    Preconditions.checkArgument(index >= 0);
+    if (index > webServer.getConnectors().length)
+      return null;
+
+    Connector c = webServer.getConnectors()[index];
+    if (c.getLocalPort() == -1) {
+      // The connector is not bounded
+      return null;
+    }
+
+    return new InetSocketAddress(c.getHost(), c.getLocalPort());
+  }
+
+  /**
+   * Set the min, max number of worker threads (simultaneous connections).
+   */
+  public void setThreads(int min, int max) {
+    QueuedThreadPool pool = (QueuedThreadPool) webServer.getThreadPool();
+    pool.setMinThreads(min);
+    pool.setMaxThreads(max);
+  }
+
+  private void initSpnego(Configuration conf, String hostName,
+      String usernameConfKey, String keytabConfKey) throws IOException {
+    Map<String, String> params = new HashMap<String, String>();
+    String principalInConf = conf.get(usernameConfKey);
+    if (principalInConf != null && !principalInConf.isEmpty()) {
+      params.put("kerberos.principal", SecurityUtil.getServerPrincipal(
+          principalInConf, hostName));
+    }
+    String httpKeytab = conf.get(keytabConfKey);
+    if (httpKeytab != null && !httpKeytab.isEmpty()) {
+      params.put("kerberos.keytab", httpKeytab);
+    }
+    params.put(AuthenticationFilter.AUTH_TYPE, "kerberos");
+  
+    defineFilter(webAppContext, SPNEGO_FILTER,
+                 AuthenticationFilter.class.getName(), params, null);
+  }
+
+  /**
+   * Start the server. Does not wait for the server to start.
+   */
+  public void start() throws IOException {
+    try {
+      try {
+        openListeners();
+        webServer.start();
+      } catch (IOException ex) {
+        LOG.info("HttpServer.start() threw a non Bind IOException", ex);
+        throw ex;
+      } catch (MultiException ex) {
+        LOG.info("HttpServer.start() threw a MultiException", ex);
+        throw ex;
+      }
+      // Make sure there is no handler failures.
+      Handler[] handlers = webServer.getHandlers();
+      for (int i = 0; i < handlers.length; i++) {
+        if (handlers[i].isFailed()) {
+          throw new IOException(
+              "Problem in starting http server. Server handlers failed");
+        }
+      }
+      // Make sure there are no errors initializing the context.
+      Throwable unavailableException = webAppContext.getUnavailableException();
+      if (unavailableException != null) {
+        // Have to stop the webserver, or else its non-daemon threads
+        // will hang forever.
+        webServer.stop();
+        throw new IOException("Unable to initialize WebAppContext",
+            unavailableException);
+      }
+    } catch (IOException e) {
+      throw e;
+    } catch (InterruptedException e) {
+      throw (IOException) new InterruptedIOException(
+          "Interrupted while starting HTTP server").initCause(e);
+    } catch (Exception e) {
+      throw new IOException("Problem starting http server", e);
+    }
+  }
+
+  private void loadListeners() {
+    for (ListenerInfo li : listeners) {
+      webServer.addConnector(li.listener);
+    }
+  }
+
+  /**
+   * Open the main listener for the server
+   * @throws Exception
+   */
+  void openListeners() throws Exception {
+    for (ListenerInfo li : listeners) {
+      Connector listener = li.listener;
+      if (!li.isManaged || li.listener.getLocalPort() != -1) {
+        // This listener is either started externally or has been bound
+        continue;
+      }
+      int port = listener.getPort();
+      while (true) {
+        // jetty has a bug where you can't reopen a listener that previously
+        // failed to open w/o issuing a close first, even if the port is changed
+        try {
+          listener.close();
+          listener.open();
+          LOG.info("Jetty bound to port " + listener.getLocalPort());
+          break;
+        } catch (BindException ex) {
+          if (port == 0 || !findPort) {
+            BindException be = new BindException("Port in use: "
+                + listener.getHost() + ":" + listener.getPort());
+            be.initCause(ex);
+            throw be;
+          }
+        }
+        // try the next port number
+        listener.setPort(++port);
+        Thread.sleep(100);
+      }
+    }
+  }
+  
+  /**
+   * stop the server
+   */
+  public void stop() throws Exception {
+    MultiException exception = null;
+    for (ListenerInfo li : listeners) {
+      if (!li.isManaged) {
+        continue;
+      }
+
+      try {
+        li.listener.close();
+      } catch (Exception e) {
+        LOG.error(
+            "Error while stopping listener for webapp"
+                + webAppContext.getDisplayName(), e);
+        exception = addMultiException(exception, e);
+      }
+    }
+
+    try {
+      // clear & stop webAppContext attributes to avoid memory leaks.
+      webAppContext.clearAttributes();
+      webAppContext.stop();
+    } catch (Exception e) {
+      LOG.error("Error while stopping web app context for webapp "
+          + webAppContext.getDisplayName(), e);
+      exception = addMultiException(exception, e);
+    }
+
+    try {
+      webServer.stop();
+    } catch (Exception e) {
+      LOG.error("Error while stopping web server for webapp "
+          + webAppContext.getDisplayName(), e);
+      exception = addMultiException(exception, e);
+    }
+
+    if (exception != null) {
+      exception.ifExceptionThrow();
+    }
+
+  }
+
+  private MultiException addMultiException(MultiException exception, Exception e) {
+    if(exception == null){
+      exception = new MultiException();
+    }
+    exception.add(e);
+    return exception;
+  }
+
+  public void join() throws InterruptedException {
+    webServer.join();
+  }
+
+  /**
+   * Test for the availability of the web server
+   * @return true if the web server is started, false otherwise
+   */
+  public boolean isAlive() {
+    return webServer != null && webServer.isStarted();
+  }
+
+  /**
+   * Return the host and port of the HttpServer, if live
+   * @return the classname and any HTTP URL
+   */
+  @Override
+  public String toString() {
+    if (listeners.size() == 0) {
+      return "Inactive HttpServer";
+    } else {
+      StringBuilder sb = new StringBuilder("HttpServer (")
+        .append(isAlive() ? STATE_DESCRIPTION_ALIVE : STATE_DESCRIPTION_NOT_LIVE).append("), listening at:");
+      for (ListenerInfo li : listeners) {
+        Connector l = li.listener;
+        sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
+      }
+      return sb.toString();
+    }
+  }
+
+  /**
+   * Checks the user has privileges to access to instrumentation servlets.
+   * <p/>
+   * If <code>hadoop.security.instrumentation.requires.admin</code> is set to FALSE
+   * (default value) it always returns TRUE.
+   * <p/>
+   * If <code>hadoop.security.instrumentation.requires.admin</code> is set to TRUE
+   * it will check that if the current user is in the admin ACLS. If the user is
+   * in the admin ACLs it returns TRUE, otherwise it returns FALSE.
+   *
+   * @param servletContext the servlet context.
+   * @param request the servlet request.
+   * @param response the servlet response.
+   * @return TRUE/FALSE based on the logic decribed above.
+   */
+  public static boolean isInstrumentationAccessAllowed(
+    ServletContext servletContext, HttpServletRequest request,
+    HttpServletResponse response) throws IOException {
+    Configuration conf =
+      (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE);
+
+    boolean access = true;
+    boolean adminAccess = conf.getBoolean(
+      CommonConfigurationKeys.HADOOP_SECURITY_INSTRUMENTATION_REQUIRES_ADMIN,
+      false);
+    if (adminAccess) {
+      access = hasAdministratorAccess(servletContext, request, response);
+    }
+    return access;
+  }
+
+  /**
+   * Does the user sending the HttpServletRequest has the administrator ACLs? If
+   * it isn't the case, response will be modified to send an error to the user.
+   * 
+   * @param servletContext
+   * @param request
+   * @param response used to send the error response if user does not have admin access.
+   * @return true if admin-authorized, false otherwise
+   * @throws IOException
+   */
+  public static boolean hasAdministratorAccess(
+      ServletContext servletContext, HttpServletRequest request,
+      HttpServletResponse response) throws IOException {
+    Configuration conf =
+        (Configuration) servletContext.getAttribute(CONF_CONTEXT_ATTRIBUTE);
+    // If there is no authorization, anybody has administrator access.
+    if (!conf.getBoolean(
+        CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false)) {
+      return true;
+    }
+
+    String remoteUser = request.getRemoteUser();
+    if (remoteUser == null) {
+      response.sendError(HttpServletResponse.SC_UNAUTHORIZED,
+                         "Unauthenticated users are not " +
+                         "authorized to access this page.");
+      return false;
+    }
+    
+    if (servletContext.getAttribute(ADMINS_ACL) != null &&
+        !userHasAdministratorAccess(servletContext, remoteUser)) {
+      response.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User "
+          + remoteUser + " is unauthorized to access this page.");
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
+   * Get the admin ACLs from the given ServletContext and check if the given
+   * user is in the ACL.
+   * 
+   * @param servletContext the context containing the admin ACL.
+   * @param remoteUser the remote user to check for.
+   * @return true if the user is present in the ACL, false if no ACL is set or
+   *         the user is not present
+   */
+  public static boolean userHasAdministratorAccess(ServletContext servletContext,
+      String remoteUser) {
+    AccessControlList adminsAcl = (AccessControlList) servletContext
+        .getAttribute(ADMINS_ACL);
+    UserGroupInformation remoteUserUGI =
+        UserGroupInformation.createRemoteUser(remoteUser);
+    return adminsAcl != null && adminsAcl.isUserAllowed(remoteUserUGI);
+  }
+
+  /**
+   * A very simple servlet to serve up a text representation of the current
+   * stack traces. It both returns the stacks to the caller and logs them.
+   * Currently the stack traces are done sequentially rather than exactly the
+   * same data.
+   */
+  public static class StackServlet extends HttpServlet {
+    private static final long serialVersionUID = -6284183679759467039L;
+
+    @Override
+    public void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws ServletException, IOException {
+      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(),
+                                                     request, response)) {
+        return;
+      }
+      response.setContentType("text/plain; charset=UTF-8");
+      PrintWriter out = response.getWriter();
+      ReflectionUtils.printThreadInfo(out, "");
+      out.close();
+      ReflectionUtils.logThreadInfo(LOG, "jsp requested", 1);      
+    }
+  }
+  
+  /**
+   * A Servlet input filter that quotes all HTML active characters in the
+   * parameter names and values. The goal is to quote the characters to make
+   * all of the servlets resistant to cross-site scripting attacks.
+   */
+  public static class QuotingInputFilter implements Filter {
+    private FilterConfig config;
+
+    public static class RequestQuoter extends HttpServletRequestWrapper {
+      private final HttpServletRequest rawRequest;
+      public RequestQuoter(HttpServletRequest rawRequest) {
+        super(rawRequest);
+        this.rawRequest = rawRequest;
+      }
+      
+      /**
+       * Return the set of parameter names, quoting each name.
+       */
+      @SuppressWarnings("unchecked")
+      @Override
+      public Enumeration<String> getParameterNames() {
+        return new Enumeration<String>() {
+          private Enumeration<String> rawIterator =
+            rawRequest.getParameterNames();
+          @Override
+          public boolean hasMoreElements() {
+            return rawIterator.hasMoreElements();
+          }
+
+          @Override
+          public String nextElement() {
+            return HtmlQuoting.quoteHtmlChars(rawIterator.nextElement());
+          }
+        };
+      }
+      
+      /**
+       * Unquote the name and quote the value.
+       */
+      @Override
+      public String getParameter(String name) {
+        return HtmlQuoting.quoteHtmlChars(rawRequest.getParameter
+                                     (HtmlQuoting.unquoteHtmlChars(name)));
+      }
+      
+      @Override
+      public String[] getParameterValues(String name) {
+        String unquoteName = HtmlQuoting.unquoteHtmlChars(name);
+        String[] unquoteValue = rawRequest.getParameterValues(unquoteName);
+        if (unquoteValue == null) {
+          return null;
+        }
+        String[] result = new String[unquoteValue.length];
+        for(int i=0; i < result.length; ++i) {
+          result[i] = HtmlQuoting.quoteHtmlChars(unquoteValue[i]);
+        }
+        return result;
+      }
+
+      @SuppressWarnings("unchecked")
+      @Override
+      public Map<String, String[]> getParameterMap() {
+        Map<String, String[]> result = new HashMap<String,String[]>();
+        Map<String, String[]> raw = rawRequest.getParameterMap();
+        for (Map.Entry<String,String[]> item: raw.entrySet()) {
+          String[] rawValue = item.getValue();
+          String[] cookedValue = new String[rawValue.length];
+          for(int i=0; i< rawValue.length; ++i) {
+            cookedValue[i] = HtmlQuoting.quoteHtmlChars(rawValue[i]);
+          }
+          result.put(HtmlQuoting.quoteHtmlChars(item.getKey()), cookedValue);
+        }
+        return result;
+      }
+      
+      /**
+       * Quote the url so that users specifying the HOST HTTP header
+       * can't inject attacks.
+       */
+      @Override
+      public StringBuffer getRequestURL(){
+        String url = rawRequest.getRequestURL().toString();
+        return new StringBuffer(HtmlQuoting.quoteHtmlChars(url));
+      }
+      
+      /**
+       * Quote the server name so that users specifying the HOST HTTP header
+       * can't inject attacks.
+       */
+      @Override
+      public String getServerName() {
+        return HtmlQuoting.quoteHtmlChars(rawRequest.getServerName());
+      }
+    }
+
+    @Override
+    public void init(FilterConfig config) throws ServletException {
+      this.config = config;
+    }
+
+    @Override
+    public void destroy() {
+    }
+
+    @Override
+    public void doFilter(ServletRequest request, 
+                         ServletResponse response,
+                         FilterChain chain
+                         ) throws IOException, ServletException {
+      HttpServletRequestWrapper quoted = 
+        new RequestQuoter((HttpServletRequest) request);
+      HttpServletResponse httpResponse = (HttpServletResponse) response;
+
+      String mime = inferMimeType(request);
+      if (mime == null) {
+        httpResponse.setContentType("text/plain; charset=utf-8");
+      } else if (mime.startsWith("text/html")) {
+        // HTML with unspecified encoding, we want to
+        // force HTML with utf-8 encoding
+        // This is to avoid the following security issue:
+        // http://openmya.hacker.jp/hasegawa/security/utf7cs.html
+        httpResponse.setContentType("text/html; charset=utf-8");
+      } else if (mime.startsWith("application/xml")) {
+        httpResponse.setContentType("text/xml; charset=utf-8");
+      }
+      chain.doFilter(quoted, httpResponse);
+    }
+
+    /**
+     * Infer the mime type for the response based on the extension of the request
+     * URI. Returns null if unknown.
+     */
+    private String inferMimeType(ServletRequest request) {
+      String path = ((HttpServletRequest)request).getRequestURI();
+      ContextHandler.SContext sContext = (ContextHandler.SContext)config.getServletContext();
+      MimeTypes mimes = sContext.getContextHandler().getMimeTypes();
+      Buffer mimeBuffer = mimes.getMimeByExtension(path);
+      return (mimeBuffer == null) ? null : mimeBuffer.toString();
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
new file mode 100644
index 0000000..911f040
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
@@ -0,0 +1,90 @@
+/**
+ *
+ * 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.IOException;
+import java.net.URI;
+
+import javax.servlet.http.HttpServlet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Create a Jetty embedded server to answer http requests. The primary goal
+ * is to serve up status information for the server.
+ * There are three contexts:
+ *   "/stacks/" -> points to stack trace
+ *   "/static/" -> points to common static files (src/hbase-webapps/static)
+ *   "/" -> the jsp server code from (src/hbase-webapps/<name>)
+ */
+@InterfaceAudience.Private
+public class InfoServer {
+  private static final String HBASE_APP_DIR = "hbase-webapps";
+  private final HttpServer httpServer;
+
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/hbase-webapps/<code>name<code>.
+   * @param name The name of the server
+   * @param bindAddress address to bind to
+   * @param port The port to use on the server
+   * @param findPort whether the server should start at the given port and
+   * increment by 1 until it finds a free port.
+   * @throws IOException e
+   */
+  public InfoServer(String name, String bindAddress, int port, boolean findPort,
+      final Configuration c)
+  throws IOException {
+    HttpServer.Builder builder = new HttpServer.Builder();
+    builder
+      .setName(name)
+      .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
+      .setAppDir(HBASE_APP_DIR).setFindPort(findPort).setConf(c);
+    String logDir = System.getProperty("hbase.log.dir");
+    if (logDir != null) {
+      builder.setLogDir(logDir);
+    }
+    this.httpServer = builder.build();
+  }
+
+  public void addServlet(String name, String pathSpec,
+          Class<? extends HttpServlet> clazz) {
+      this.httpServer.addServlet(name, pathSpec, clazz);
+  }
+
+  public void setAttribute(String name, Object value) {
+    this.httpServer.setAttribute(name, value);
+  }
+
+  public void start() throws IOException {
+    this.httpServer.start();
+  }
+
+  @Deprecated
+  public int getPort() {
+    return this.httpServer.getPort();
+  }
+
+  public void stop() throws Exception {
+    this.httpServer.stop();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.java
new file mode 100644
index 0000000..fffbd8e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.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 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.HttpServletResponse;
+import java.io.IOException;
+
+public class NoCacheFilter implements Filter {
+
+  @Override
+  public void init(FilterConfig filterConfig) throws ServletException {
+  }
+
+  @Override
+  public void doFilter(ServletRequest req, ServletResponse res,
+                       FilterChain chain)
+    throws IOException, ServletException {
+    HttpServletResponse httpRes = (HttpServletResponse) res;
+    httpRes.setHeader("Cache-Control", "no-cache");
+    long now = System.currentTimeMillis();
+    httpRes.addDateHeader("Expires", now);
+    httpRes.addDateHeader("Date", now);
+    httpRes.addHeader("Pragma", "no-cache");
+    chain.doFilter(req, res);
+  }
+
+  @Override
+  public void destroy() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/main/java/org/apache/hadoop/hbase/http/ServerConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/ServerConfigurationKeys.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/ServerConfigurationKeys.java
new file mode 100644
index 0000000..80026ff
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/ServerConfigurationKeys.java
@@ -0,0 +1,47 @@
+/**
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/** 
+ * This interface contains constants for configuration keys used
+ * in the hbase http server code.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface ServerConfigurationKeys {
+
+  /** Enable/Disable ssl for http server */
+  public static final String HBASE_SSL_ENABLED_KEY = "hbase.ssl.enabled";
+
+  public static final boolean HBASE_SSL_ENABLED_DEFAULT = false;
+
+  /** Enable/Disable aliases serving from jetty */
+  public static final String HBASE_JETTY_LOGS_SERVE_ALIASES =
+      "hbase.jetty.logs.serve.aliases";
+
+  public static final boolean DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES =
+      true;
+
+  public static final String HBASE_HTTP_STATIC_USER = "hbase.http.staticuser.user";
+
+  public static final String DEFAULT_HBASE_HTTP_STATIC_USER = "dr.stack";
+
+}


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

Posted by st...@apache.org.
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());
+  }
+}


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

Posted by st...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java
new file mode 100644
index 0000000..b92eabc
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java
@@ -0,0 +1,613 @@
+/**
+ * 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.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.HttpServer.QuotingInputFilter.RequestQuoter;
+import org.apache.hadoop.hbase.http.resource.JerseyResource;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.Groups;
+import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+import org.mockito.internal.util.reflection.Whitebox;
+import org.mortbay.jetty.Connector;
+import org.mortbay.util.ajax.JSON;
+
+@Category(SmallTests.class)
+public class TestHttpServer extends HttpServerFunctionalTest {
+  static final Log LOG = LogFactory.getLog(TestHttpServer.class);
+  private static HttpServer server;
+  private static URL baseUrl;
+  private static final int MAX_THREADS = 10;
+  
+  @SuppressWarnings("serial")
+  public static class EchoMapServlet extends HttpServlet {
+    @SuppressWarnings("unchecked")
+    @Override
+    public void doGet(HttpServletRequest request, 
+                      HttpServletResponse response
+                      ) throws ServletException, IOException {
+      PrintWriter out = response.getWriter();
+      Map<String, String[]> params = request.getParameterMap();
+      SortedSet<String> keys = new TreeSet<String>(params.keySet());
+      for(String key: keys) {
+        out.print(key);
+        out.print(':');
+        String[] values = params.get(key);
+        if (values.length > 0) {
+          out.print(values[0]);
+          for(int i=1; i < values.length; ++i) {
+            out.print(',');
+            out.print(values[i]);
+          }
+        }
+        out.print('\n');
+      }
+      out.close();
+    }    
+  }
+
+  @SuppressWarnings("serial")
+  public static class EchoServlet extends HttpServlet {
+    @SuppressWarnings("unchecked")
+    @Override
+    public void doGet(HttpServletRequest request, 
+                      HttpServletResponse response
+                      ) throws ServletException, IOException {
+      PrintWriter out = response.getWriter();
+      SortedSet<String> sortedKeys = new TreeSet<String>();
+      Enumeration<String> keys = request.getParameterNames();
+      while(keys.hasMoreElements()) {
+        sortedKeys.add(keys.nextElement());
+      }
+      for(String key: sortedKeys) {
+        out.print(key);
+        out.print(':');
+        out.print(request.getParameter(key));
+        out.print('\n');
+      }
+      out.close();
+    }    
+  }
+
+  @SuppressWarnings("serial")
+  public static class LongHeaderServlet extends HttpServlet {
+    @Override
+    public void doGet(HttpServletRequest request,
+                      HttpServletResponse response
+    ) throws ServletException, IOException {
+      Assert.assertEquals(63 * 1024, request.getHeader("longheader").length());
+      response.setStatus(HttpServletResponse.SC_OK);
+    }
+  }
+
+  @SuppressWarnings("serial")
+  public static class HtmlContentServlet extends HttpServlet {
+    @Override
+    public void doGet(HttpServletRequest request, 
+                      HttpServletResponse response
+                      ) throws ServletException, IOException {
+      response.setContentType("text/html");
+      PrintWriter out = response.getWriter();
+      out.print("hello world");
+      out.close();
+    }
+  }
+
+  @BeforeClass public static void setup() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(HttpServer.HTTP_MAX_THREADS, 10);
+    server = createTestServer(conf);
+    server.addServlet("echo", "/echo", EchoServlet.class);
+    server.addServlet("echomap", "/echomap", EchoMapServlet.class);
+    server.addServlet("htmlcontent", "/htmlcontent", HtmlContentServlet.class);
+    server.addServlet("longheader", "/longheader", LongHeaderServlet.class);
+    server.addJerseyResourcePackage(
+        JerseyResource.class.getPackage().getName(), "/jersey/*");
+    server.start();
+    baseUrl = getServerURL(server);
+    LOG.info("HTTP server started: "+ baseUrl);
+  }
+  
+  @AfterClass public static void cleanup() throws Exception {
+    server.stop();
+  }
+  
+  /** Test the maximum number of threads cannot be exceeded. */
+  @Test public void testMaxThreads() throws Exception {
+    int clientThreads = MAX_THREADS * 10;
+    Executor executor = Executors.newFixedThreadPool(clientThreads);
+    // Run many clients to make server reach its maximum number of threads
+    final CountDownLatch ready = new CountDownLatch(clientThreads);
+    final CountDownLatch start = new CountDownLatch(1);
+    for (int i = 0; i < clientThreads; i++) {
+      executor.execute(new Runnable() {
+        @Override
+        public void run() {
+          ready.countDown();
+          try {
+            start.await();
+            assertEquals("a:b\nc:d\n",
+                         readOutput(new URL(baseUrl, "/echo?a=b&c=d")));
+            int serverThreads = server.webServer.getThreadPool().getThreads();
+            assertTrue("More threads are started than expected, Server Threads count: "
+                    + serverThreads, serverThreads <= MAX_THREADS);
+            System.out.println("Number of threads = " + serverThreads +
+                " which is less or equal than the max = " + MAX_THREADS);
+          } catch (Exception e) {
+            // do nothing
+          }
+        }
+      });
+    }
+    // Start the client threads when they are all ready
+    ready.await();
+    start.countDown();
+  }
+  
+  @Test public void testEcho() throws Exception {
+    assertEquals("a:b\nc:d\n", 
+                 readOutput(new URL(baseUrl, "/echo?a=b&c=d")));
+    assertEquals("a:b\nc&lt;:d\ne:&gt;\n", 
+                 readOutput(new URL(baseUrl, "/echo?a=b&c<=d&e=>")));    
+  }
+  
+  /** Test the echo map servlet that uses getParameterMap. */
+  @Test public void testEchoMap() throws Exception {
+    assertEquals("a:b\nc:d\n", 
+                 readOutput(new URL(baseUrl, "/echomap?a=b&c=d")));
+    assertEquals("a:b,&gt;\nc&lt;:d\n", 
+                 readOutput(new URL(baseUrl, "/echomap?a=b&c<=d&a=>")));
+  }
+
+  /** 
+   *  Test that verifies headers can be up to 64K long. 
+   *  The test adds a 63K header leaving 1K for other headers.
+   *  This is because the header buffer setting is for ALL headers,
+   *  names and values included. */
+  @Test public void testLongHeader() throws Exception {
+    URL url = new URL(baseUrl, "/longheader");
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0 ; i < 63 * 1024; i++) {
+      sb.append("a");
+    }
+    conn.setRequestProperty("longheader", sb.toString());
+    assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+  }
+
+  @Test
+  @Ignore
+  public void testContentTypes() throws Exception {
+    // Static CSS files should have text/css
+    URL cssUrl = new URL(baseUrl, "/static/test.css");
+    HttpURLConnection conn = (HttpURLConnection)cssUrl.openConnection();
+    conn.connect();
+    assertEquals(200, conn.getResponseCode());
+    assertEquals("text/css", conn.getContentType());
+
+    // Servlets should have text/plain with proper encoding by default
+    URL servletUrl = new URL(baseUrl, "/echo?a=b");
+    conn = (HttpURLConnection)servletUrl.openConnection();
+    conn.connect();
+    assertEquals(200, conn.getResponseCode());
+    assertEquals("text/plain; charset=utf-8", conn.getContentType());
+
+    // We should ignore parameters for mime types - ie a parameter
+    // ending in .css should not change mime type
+    servletUrl = new URL(baseUrl, "/echo?a=b.css");
+    conn = (HttpURLConnection)servletUrl.openConnection();
+    conn.connect();
+    assertEquals(200, conn.getResponseCode());
+    assertEquals("text/plain; charset=utf-8", conn.getContentType());
+
+    // Servlets that specify text/html should get that content type
+    servletUrl = new URL(baseUrl, "/htmlcontent");
+    conn = (HttpURLConnection)servletUrl.openConnection();
+    conn.connect();
+    assertEquals(200, conn.getResponseCode());
+    assertEquals("text/html; charset=utf-8", conn.getContentType());
+
+    // JSPs should default to text/html with utf8
+    servletUrl = new URL(baseUrl, "/testjsp.jsp");
+    conn = (HttpURLConnection)servletUrl.openConnection();
+    conn.connect();
+    assertEquals(200, conn.getResponseCode());
+    assertEquals("text/html; charset=utf-8", conn.getContentType());
+  }
+
+  /**
+   * Dummy filter that mimics as an authentication filter. Obtains user identity
+   * from the request parameter user.name. Wraps around the request so that
+   * request.getRemoteUser() returns the user identity.
+   * 
+   */
+  public static class DummyServletFilter implements Filter {
+    @Override
+    public void destroy() { }
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response,
+        FilterChain filterChain) throws IOException, ServletException {
+      final String userName = request.getParameter("user.name");
+      ServletRequest requestModified =
+        new HttpServletRequestWrapper((HttpServletRequest) request) {
+        @Override
+        public String getRemoteUser() {
+          return userName;
+        }
+      };
+      filterChain.doFilter(requestModified, response);
+    }
+
+    @Override
+    public void init(FilterConfig arg0) throws ServletException { }
+  }
+
+  /**
+   * FilterInitializer that initialized the DummyFilter.
+   *
+   */
+  public static class DummyFilterInitializer extends FilterInitializer {
+    public DummyFilterInitializer() {
+    }
+
+    @Override
+    public void initFilter(FilterContainer container, Configuration conf) {
+      container.addFilter("DummyFilter", DummyServletFilter.class.getName(), null);
+    }
+  }
+
+  /**
+   * Access a URL and get the corresponding return Http status code. The URL
+   * will be accessed as the passed user, by sending user.name request
+   * parameter.
+   * 
+   * @param urlstring
+   * @param userName
+   * @return
+   * @throws IOException
+   */
+  static int getHttpStatusCode(String urlstring, String userName)
+      throws IOException {
+    URL url = new URL(urlstring + "?user.name=" + userName);
+    System.out.println("Accessing " + url + " as user " + userName);
+    HttpURLConnection connection = (HttpURLConnection)url.openConnection();
+    connection.connect();
+    return connection.getResponseCode();
+  }
+
+  /**
+   * Custom user->group mapping service.
+   */
+  public static class MyGroupsProvider extends ShellBasedUnixGroupsMapping {
+    static Map<String, List<String>> mapping = new HashMap<String, List<String>>();
+
+    static void clearMapping() {
+      mapping.clear();
+    }
+
+    @Override
+    public List<String> getGroups(String user) throws IOException {
+      return mapping.get(user);
+    }
+  }
+
+  /**
+   * Verify the access for /logs, /stacks, /conf, /logLevel and /metrics
+   * servlets, when authentication filters are set, but authorization is not
+   * enabled.
+   * @throws Exception 
+   */
+  @Test
+  @Ignore
+  public void testDisabledAuthorizationOfDefaultServlets() throws Exception {
+
+    Configuration conf = new Configuration();
+
+    // Authorization is disabled by default
+    conf.set(HttpServer.FILTER_INITIALIZERS_PROPERTY,
+        DummyFilterInitializer.class.getName());
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+        MyGroupsProvider.class.getName());
+    Groups.getUserToGroupsMappingService(conf);
+    MyGroupsProvider.clearMapping();
+    MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
+    MyGroupsProvider.mapping.put("userB", Arrays.asList("groupB"));
+
+    HttpServer myServer = new HttpServer.Builder().setName("test")
+        .addEndpoint(new URI("http://localhost:0")).setFindPort(true).build();
+    myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
+    myServer.start();
+    String serverURL = "http://" + NetUtils.getHostPortString(myServer.getConnectorAddress(0)) + "/";
+    for (String servlet : new String[] { "conf", "logs", "stacks",
+        "logLevel", "metrics" }) {
+      for (String user : new String[] { "userA", "userB" }) {
+        assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(serverURL
+            + servlet, user));
+      }
+    }
+    myServer.stop();
+  }
+
+  /**
+   * Verify the administrator access for /logs, /stacks, /conf, /logLevel and
+   * /metrics servlets.
+   * 
+   * @throws Exception
+   */
+  @Test
+  @Ignore
+  public void testAuthorizationOfDefaultServlets() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
+        true);
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_INSTRUMENTATION_REQUIRES_ADMIN,
+        true);
+    conf.set(HttpServer.FILTER_INITIALIZERS_PROPERTY,
+        DummyFilterInitializer.class.getName());
+
+    conf.set(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
+        MyGroupsProvider.class.getName());
+    Groups.getUserToGroupsMappingService(conf);
+    MyGroupsProvider.clearMapping();
+    MyGroupsProvider.mapping.put("userA", Arrays.asList("groupA"));
+    MyGroupsProvider.mapping.put("userB", Arrays.asList("groupB"));
+    MyGroupsProvider.mapping.put("userC", Arrays.asList("groupC"));
+    MyGroupsProvider.mapping.put("userD", Arrays.asList("groupD"));
+    MyGroupsProvider.mapping.put("userE", Arrays.asList("groupE"));
+
+    HttpServer myServer = new HttpServer.Builder().setName("test")
+        .addEndpoint(new URI("http://localhost:0")).setFindPort(true).setConf(conf)
+        .setACL(new AccessControlList("userA,userB groupC,groupD")).build();
+    myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
+    myServer.start();
+
+    String serverURL = "http://"
+        + NetUtils.getHostPortString(myServer.getConnectorAddress(0)) + "/";
+    for (String servlet : new String[] { "conf", "logs", "stacks",
+        "logLevel", "metrics" }) {
+      for (String user : new String[] { "userA", "userB", "userC", "userD" }) {
+        assertEquals(HttpURLConnection.HTTP_OK, getHttpStatusCode(serverURL
+            + servlet, user));
+      }
+      assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, getHttpStatusCode(
+          serverURL + servlet, "userE"));
+    }
+    myServer.stop();
+  }
+  
+  @Test
+  public void testRequestQuoterWithNull() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    Mockito.doReturn(null).when(request).getParameterValues("dummy");
+    RequestQuoter requestQuoter = new RequestQuoter(request);
+    String[] parameterValues = requestQuoter.getParameterValues("dummy");
+    Assert.assertEquals("It should return null "
+        + "when there are no values for the parameter", null, parameterValues);
+  }
+
+  @Test
+  public void testRequestQuoterWithNotNull() throws Exception {
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    String[] values = new String[] { "abc", "def" };
+    Mockito.doReturn(values).when(request).getParameterValues("dummy");
+    RequestQuoter requestQuoter = new RequestQuoter(request);
+    String[] parameterValues = requestQuoter.getParameterValues("dummy");
+    Assert.assertTrue("It should return Parameter Values", Arrays.equals(
+        values, parameterValues));
+  }
+
+  @SuppressWarnings("unchecked")
+  private static Map<String, Object> parse(String jsonString) {
+    return (Map<String, Object>)JSON.parse(jsonString);
+  }
+
+  @Test public void testJersey() throws Exception {
+    LOG.info("BEGIN testJersey()");
+    final String js = readOutput(new URL(baseUrl, "/jersey/foo?op=bar"));
+    final Map<String, Object> m = parse(js);
+    LOG.info("m=" + m);
+    assertEquals("foo", m.get(JerseyResource.PATH));
+    assertEquals("bar", m.get(JerseyResource.OP));
+    LOG.info("END testJersey()");
+  }
+
+  @Test
+  public void testHasAdministratorAccess() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, false);
+    ServletContext context = Mockito.mock(ServletContext.class);
+    Mockito.when(context.getAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE)).thenReturn(conf);
+    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(null);
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    Mockito.when(request.getRemoteUser()).thenReturn(null);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    //authorization OFF
+    Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
+
+    //authorization ON & user NULL
+    response = Mockito.mock(HttpServletResponse.class);
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
+    Assert.assertFalse(HttpServer.hasAdministratorAccess(context, request, response));
+    Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED), Mockito.anyString());
+
+    //authorization ON & user NOT NULL & ACLs NULL
+    response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(request.getRemoteUser()).thenReturn("foo");
+    Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
+
+    //authorization ON & user NOT NULL & ACLs NOT NULL & user not in ACLs
+    response = Mockito.mock(HttpServletResponse.class);
+    AccessControlList acls = Mockito.mock(AccessControlList.class);
+    Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(false);
+    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
+    Assert.assertFalse(HttpServer.hasAdministratorAccess(context, request, response));
+    Mockito.verify(response).sendError(Mockito.eq(HttpServletResponse.SC_UNAUTHORIZED), Mockito.anyString());
+
+    //authorization ON & user NOT NULL & ACLs NOT NULL & user in in ACLs
+    response = Mockito.mock(HttpServletResponse.class);
+    Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(true);
+    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
+    Assert.assertTrue(HttpServer.hasAdministratorAccess(context, request, response));
+
+  }
+
+  @Test
+  public void testRequiresAuthorizationAccess() throws Exception {
+    Configuration conf = new Configuration();
+    ServletContext context = Mockito.mock(ServletContext.class);
+    Mockito.when(context.getAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE)).thenReturn(conf);
+    HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
+    HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
+
+    //requires admin access to instrumentation, FALSE by default
+    Assert.assertTrue(HttpServer.isInstrumentationAccessAllowed(context, request, response));
+
+    //requires admin access to instrumentation, TRUE
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_INSTRUMENTATION_REQUIRES_ADMIN, true);
+    conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION, true);
+    AccessControlList acls = Mockito.mock(AccessControlList.class);
+    Mockito.when(acls.isUserAllowed(Mockito.<UserGroupInformation>any())).thenReturn(false);
+    Mockito.when(context.getAttribute(HttpServer.ADMINS_ACL)).thenReturn(acls);
+    Assert.assertFalse(HttpServer.isInstrumentationAccessAllowed(context, request, response));
+  }
+
+  @Test public void testBindAddress() throws Exception {
+    checkBindAddress("localhost", 0, false).stop();
+    // hang onto this one for a bit more testing
+    HttpServer myServer = checkBindAddress("localhost", 0, false);
+    HttpServer myServer2 = null;
+    try { 
+      int port = myServer.getConnectorAddress(0).getPort();
+      // it's already in use, true = expect a higher port
+      myServer2 = checkBindAddress("localhost", port, true);
+      // try to reuse the port
+      port = myServer2.getConnectorAddress(0).getPort();
+      myServer2.stop();
+      assertNull(myServer2.getConnectorAddress(0)); // not bound
+      myServer2.openListeners();
+      assertEquals(port, myServer2.getConnectorAddress(0).getPort()); // expect same port
+    } finally {
+      myServer.stop();
+      if (myServer2 != null) {
+        myServer2.stop();
+      }
+    }
+  }
+  
+  private HttpServer checkBindAddress(String host, int port, boolean findPort)
+      throws Exception {
+    HttpServer server = createServer(host, port);
+    try {
+      // not bound, ephemeral should return requested port (0 for ephemeral)
+      List<?> listeners = (List<?>) Whitebox.getInternalState(server,
+          "listeners");
+      Connector listener = (Connector) Whitebox.getInternalState(
+          listeners.get(0), "listener");
+
+      assertEquals(port, listener.getPort());
+      // verify hostname is what was given
+      server.openListeners();
+      assertEquals(host, server.getConnectorAddress(0).getHostName());
+
+      int boundPort = server.getConnectorAddress(0).getPort();
+      if (port == 0) {
+        assertTrue(boundPort != 0); // ephemeral should now return bound port
+      } else if (findPort) {
+        assertTrue(boundPort > port);
+        // allow a little wiggle room to prevent random test failures if
+        // some consecutive ports are already in use
+        assertTrue(boundPort - port < 8);
+      }
+    } catch (Exception e) {
+      server.stop();
+      throw e;
+    }
+    return server;
+  }
+
+  @Test
+  public void testNoCacheHeader() throws Exception {
+    URL url = new URL(baseUrl, "/echo?a=b&c=d");
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+    assertEquals("no-cache", conn.getHeaderField("Cache-Control"));
+    assertEquals("no-cache", conn.getHeaderField("Pragma"));
+    assertNotNull(conn.getHeaderField("Expires"));
+    assertNotNull(conn.getHeaderField("Date"));
+    assertEquals(conn.getHeaderField("Expires"), conn.getHeaderField("Date"));
+  }
+
+  /**
+   * HTTPServer.Builder should proceed if a external connector is available.
+   */
+  @Test
+  public void testHttpServerBuilderWithExternalConnector() throws Exception {
+    Connector c = mock(Connector.class);
+    doReturn("localhost").when(c).getHost();
+    HttpServer s = new HttpServer.Builder().setName("test").setConnector(c)
+        .build();
+    s.stop();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.java
new file mode 100644
index 0000000..2d139e1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.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 org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.HttpRequestLogAppender;
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.apache.log4j.Logger;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestHttpServerLifecycle extends HttpServerFunctionalTest {
+
+  /**
+   * Check that a server is alive by probing the {@link HttpServer#isAlive()} method
+   * and the text of its toString() description
+   * @param server server
+   */
+  private void assertAlive(HttpServer server) {
+    assertTrue("Server is not alive", server.isAlive());
+    assertToStringContains(server, HttpServer.STATE_DESCRIPTION_ALIVE);
+  }
+
+  private void assertNotLive(HttpServer server) {
+    assertTrue("Server should not be live", !server.isAlive());
+    assertToStringContains(server, HttpServer.STATE_DESCRIPTION_NOT_LIVE);
+  }
+
+  /**
+   * Test that the server is alive once started
+   *
+   * @throws Throwable on failure
+   */
+  @Test public void testCreatedServerIsNotAlive() throws Throwable {
+    HttpServer server = createTestServer();
+    assertNotLive(server);
+  }
+
+  @Test public void testStopUnstartedServer() throws Throwable {
+    HttpServer server = createTestServer();
+    stop(server);
+  }
+
+  /**
+   * Test that the server is alive once started
+   *
+   * @throws Throwable on failure
+   */
+  @Test
+  public void testStartedServerIsAlive() throws Throwable {
+    HttpServer server = null;
+    server = createTestServer();
+    assertNotLive(server);
+    server.start();
+    assertAlive(server);
+    stop(server);
+  }
+
+  /**
+   * Test that the server with request logging enabled
+   *
+   * @throws Throwable on failure
+   */
+  @Test
+  public void testStartedServerWithRequestLog() throws Throwable {
+    HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender();
+    requestLogAppender.setName("httprequestlog");
+    requestLogAppender.setFilename(System.getProperty("test.build.data", "/tmp/")
+        + "jetty-name-yyyy_mm_dd.log");
+    Logger.getLogger(HttpServer.class.getName() + ".test").addAppender(requestLogAppender);
+    HttpServer server = null;
+    server = createTestServer();
+    assertNotLive(server);
+    server.start();
+    assertAlive(server);
+    stop(server);
+    Logger.getLogger(HttpServer.class.getName() + ".test").removeAppender(requestLogAppender);
+  }
+
+  /**
+   * Assert that the result of {@link HttpServer#toString()} contains the specific text
+   * @param server server to examine
+   * @param text text to search for
+   */
+  private void assertToStringContains(HttpServer server, String text) {
+    String description = server.toString();
+    assertTrue("Did not find \"" + text + "\" in \"" + description + "\"",
+               description.contains(text));
+  }
+
+  /**
+   * Test that the server is not alive once stopped
+   *
+   * @throws Throwable on failure
+   */
+  @Test public void testStoppedServerIsNotAlive() throws Throwable {
+    HttpServer server = createAndStartTestServer();
+    assertAlive(server);
+    stop(server);
+    assertNotLive(server);
+  }
+
+  /**
+   * Test that the server is not alive once stopped
+   *
+   * @throws Throwable on failure
+   */
+  @Test public void testStoppingTwiceServerIsAllowed() throws Throwable {
+    HttpServer server = createAndStartTestServer();
+    assertAlive(server);
+    stop(server);
+    assertNotLive(server);
+    stop(server);
+    assertNotLive(server);
+  }
+
+  /**
+   * Test that the server is alive once started
+   * 
+   * @throws Throwable
+   *           on failure
+   */
+  @Test
+  public void testWepAppContextAfterServerStop() throws Throwable {
+    HttpServer server = null;
+    String key = "test.attribute.key";
+    String value = "test.attribute.value";
+    server = createTestServer();
+    assertNotLive(server);
+    server.start();
+    server.setAttribute(key, value);
+    assertAlive(server);
+    assertEquals(value, server.getAttribute(key));
+    stop(server);
+    assertNull("Server context should have cleared", server.getAttribute(key));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerWebapps.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerWebapps.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerWebapps.java
new file mode 100644
index 0000000..ce6da2b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerWebapps.java
@@ -0,0 +1,68 @@
+/**
+ * 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.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.HttpServer;
+
+import java.io.FileNotFoundException;
+
+/**
+ * Test webapp loading
+ */
+@Category(SmallTests.class)
+public class TestHttpServerWebapps extends HttpServerFunctionalTest {
+  private static final Log log = LogFactory.getLog(TestHttpServerWebapps.class);
+
+  /**
+   * Test that the test server is loadable on the classpath
+   * @throws Throwable if something went wrong
+   */
+  @Test
+  public void testValidServerResource() throws Throwable {
+    HttpServer server = null;
+    try {
+      server = createServer("test");
+    } finally {
+      stop(server);
+    }
+  }
+
+  /**
+   * Test that an invalid webapp triggers an exception
+   * @throws Throwable if something went wrong
+   */
+  @Test
+  public void testMissingServerResource() throws Throwable {
+    try {
+      HttpServer server = createServer("NoSuchWebapp");
+      //should not have got here.
+      //close the server
+      String serverDescription = server.toString();
+      stop(server);
+      fail("Expected an exception, got " + serverDescription);
+    } catch (FileNotFoundException expected) {
+      log.debug("Expected exception " + expected, expected);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java
new file mode 100644
index 0000000..9fade20
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java
@@ -0,0 +1,157 @@
+/**
+ * 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 TestPathFilter 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.addFilter("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 testPathSpecFilters() throws Exception {
+    Configuration conf = new Configuration();
+    
+    //start a http server with CountingFilter
+    conf.set(HttpServer.FILTER_INITIALIZERS_PROPERTY,
+        RecordingFilter.Initializer.class.getName());
+    String[] pathSpecs = { "/path", "/path/*" };
+    HttpServer http = createTestServer(conf, pathSpecs);
+    http.start();
+
+    final String baseURL = "/path";
+    final String baseSlashURL = "/path/";
+    final String addedURL = "/path/nodes";
+    final String addedSlashURL = "/path/nodes/";
+    final String longURL = "/path/nodes/foo/job";
+    final String rootURL = "/";
+    final String allURL = "/*";
+
+    final String[] filteredUrls = {baseURL, baseSlashURL, addedURL, 
+        addedSlashURL, longURL};
+    final String[] notFilteredUrls = {rootURL, allURL};
+
+    // access the urls and verify our paths specs got added to the 
+    // filters
+    final String prefix = "http://"
+        + NetUtils.getHostPortString(http.getConnectorAddress(0));
+    try {
+      for(int i = 0; i < filteredUrls.length; i++) {
+        access(prefix + filteredUrls[i]);
+      }
+      for(int i = 0; i < notFilteredUrls.length; i++) {
+        access(prefix + notFilteredUrls[i]);
+      }
+    } finally {
+      http.stop();
+    }
+
+    LOG.info("RECORDS = " + RECORDS);
+    
+    //verify records
+    for(int i = 0; i < filteredUrls.length; i++) {
+      assertTrue(RECORDS.remove(filteredUrls[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/TestSSLHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java
new file mode 100644
index 0000000..8244ba9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java
@@ -0,0 +1,121 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URL;
+
+import javax.net.ssl.HttpsURLConnection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * This testcase issues SSL certificates configures the HttpServer to serve
+ * HTTPS using the created certficates and calls an echo servlet using the
+ * corresponding HTTPS URL.
+ */
+@Category(SmallTests.class)
+public class TestSSLHttpServer extends HttpServerFunctionalTest {
+  private static final String BASEDIR = System.getProperty("test.build.dir",
+      "target/test-dir") + "/" + TestSSLHttpServer.class.getSimpleName();
+
+  private static final Log LOG = LogFactory.getLog(TestSSLHttpServer.class);
+  private static Configuration conf;
+  private static HttpServer server;
+  private static URL baseUrl;
+  private static String keystoresDir;
+  private static String sslConfDir;
+  private static SSLFactory clientSslFactory;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    conf = new Configuration();
+    conf.setInt(HttpServer.HTTP_MAX_THREADS, 10);
+
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+    keystoresDir = new File(BASEDIR).getAbsolutePath();
+    sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
+
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
+    Configuration sslConf = new Configuration(false);
+    sslConf.addResource("ssl-server.xml");
+    sslConf.addResource("ssl-client.xml");
+
+    clientSslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, sslConf);
+    clientSslFactory.init();
+
+    server = new HttpServer.Builder()
+        .setName("test")
+        .addEndpoint(new URI("https://localhost"))
+        .setConf(conf)
+        .keyPassword(sslConf.get("ssl.server.keystore.keypassword"))
+        .keyStore(sslConf.get("ssl.server.keystore.location"),
+            sslConf.get("ssl.server.keystore.password"),
+            sslConf.get("ssl.server.keystore.type", "jks"))
+        .trustStore(sslConf.get("ssl.server.truststore.location"),
+            sslConf.get("ssl.server.truststore.password"),
+            sslConf.get("ssl.server.truststore.type", "jks")).build();
+    server.addServlet("echo", "/echo", TestHttpServer.EchoServlet.class);
+    server.start();
+    baseUrl = new URL("https://"
+        + NetUtils.getHostPortString(server.getConnectorAddress(0)));
+    LOG.info("HTTP server started: " + baseUrl);
+  }
+
+  @AfterClass
+  public static void cleanup() throws Exception {
+    server.stop();
+    FileUtil.fullyDelete(new File(BASEDIR));
+    KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
+    clientSslFactory.destroy();
+  }
+
+  @Test
+  public void testEcho() throws Exception {
+    assertEquals("a:b\nc:d\n", readOut(new URL(baseUrl, "/echo?a=b&c=d")));
+    assertEquals("a:b\nc&lt;:d\ne:&gt;\n", readOut(new URL(baseUrl,
+        "/echo?a=b&c<=d&e=>")));
+  }
+
+  private static String readOut(URL url) throws Exception {
+    HttpsURLConnection conn = (HttpsURLConnection) url.openConnection();
+    conn.setSSLSocketFactory(clientSslFactory.createSSLSocketFactory());
+    InputStream in = conn.getInputStream();
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    IOUtils.copyBytes(in, out, 1024);
+    return out.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java
new file mode 100644
index 0000000..9864827
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java
@@ -0,0 +1,204 @@
+/**
+ * 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.Random;
+
+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.GenericTestUtils;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestServletFilter extends HttpServerFunctionalTest {
+  static final Log LOG = LogFactory.getLog(HttpServer.class);
+  static volatile String uri = null; 
+
+  /** A very simple filter which record the uri filtered. */
+  static public class SimpleFilter implements Filter {
+    private FilterConfig filterConfig = null;
+
+    @Override
+    public void init(FilterConfig filterConfig) throws ServletException {
+      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;
+
+      uri = ((HttpServletRequest)request).getRequestURI();
+      LOG.info("filtering " + uri);
+      chain.doFilter(request, response);
+    }
+
+    /** Configuration for the filter */
+    static public class Initializer extends FilterInitializer {
+      public Initializer() {}
+
+      @Override
+      public void initFilter(FilterContainer container, Configuration conf) {
+        container.addFilter("simple", SimpleFilter.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,
+        SimpleFilter.Initializer.class.getName());
+    HttpServer http = createTestServer(conf);
+    http.start();
+
+    final String fsckURL = "/fsck";
+    final String stacksURL = "/stacks";
+    final String ajspURL = "/a.jsp";
+    final String logURL = "/logs/a.log";
+    final String hadooplogoURL = "/static/hadoop-logo.jpg";
+    
+    final String[] urls = {fsckURL, stacksURL, ajspURL, logURL, hadooplogoURL};
+    final Random ran = new Random();
+    final int[] sequence = new int[50];
+
+    //generate a random sequence and update counts 
+    for(int i = 0; i < sequence.length; i++) {
+      sequence[i] = ran.nextInt(urls.length);
+    }
+
+    //access the urls as the sequence
+    final String prefix = "http://"
+        + NetUtils.getHostPortString(http.getConnectorAddress(0));
+    try {
+      for(int i = 0; i < sequence.length; i++) {
+        access(prefix + urls[sequence[i]]);
+
+        //make sure everything except fsck get filtered
+        if (sequence[i] == 0) {
+          assertEquals(null, uri);
+        } else {
+          assertEquals(urls[sequence[i]], uri);
+          uri = null;
+        }
+      }
+    } finally {
+      http.stop();
+    }
+  }
+  
+  static public class ErrorFilter extends SimpleFilter {
+    @Override
+    public void init(FilterConfig arg0) throws ServletException {
+      throw new ServletException("Throwing the exception from Filter init");
+    }
+
+    /** Configuration for the filter */
+    static public class Initializer extends FilterInitializer {
+      public Initializer() {
+      }
+
+      @Override
+      public void initFilter(FilterContainer container, Configuration conf) {
+        container.addFilter("simple", ErrorFilter.class.getName(), null);
+      }
+    }
+  }
+
+  @Test
+  public void testServletFilterWhenInitThrowsException() throws Exception {
+    Configuration conf = new Configuration();
+    // start a http server with ErrorFilter
+    conf.set(HttpServer.FILTER_INITIALIZERS_PROPERTY,
+        ErrorFilter.Initializer.class.getName());
+    HttpServer http = createTestServer(conf);
+    try {
+      http.start();
+      fail("expecting exception");
+    } catch (IOException e) {
+      assertTrue( e.getMessage().contains("Problem in starting http server. Server handlers failed"));
+    }
+  }
+  
+  /**
+   * Similar to the above test case, except that it uses a different API to add the
+   * filter. Regression test for HADOOP-8786.
+   */
+  @Test
+  public void testContextSpecificServletFilterWhenInitThrowsException()
+      throws Exception {
+    Configuration conf = new Configuration();
+    HttpServer http = createTestServer(conf);
+    HttpServer.defineFilter(http.webAppContext,
+        "ErrorFilter", ErrorFilter.class.getName(),
+        null, null);
+    try {
+      http.start();
+      fail("expecting exception");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Unable to initialize WebAppContext", e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java
new file mode 100644
index 0000000..f805cd6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java
@@ -0,0 +1,116 @@
+/**
+ * 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.StringReader;
+import java.io.StringWriter;
+import java.util.Map;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.conf.ConfServlet;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mortbay.util.ajax.JSON;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
+
+/**
+ * Basic test case that the ConfServlet can write configuration
+ * to its output in XML and JSON format.
+ */
+@Category(SmallTests.class)
+public class TestConfServlet extends TestCase {
+  private static final String TEST_KEY = "testconfservlet.key";
+  private static final String TEST_VAL = "testval";
+
+  private Configuration getTestConf() {
+    Configuration testConf = new Configuration();
+    testConf.set(TEST_KEY, TEST_VAL);
+    return testConf;
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testWriteJson() throws Exception {
+    StringWriter sw = new StringWriter();
+    ConfServlet.writeResponse(getTestConf(), sw, "json");
+    String json = sw.toString();
+    boolean foundSetting = false;
+    Object parsed = JSON.parse(json);
+    Object[] properties = ((Map<String, Object[]>)parsed).get("properties");
+    for (Object o : properties) {
+      Map<String, Object> propertyInfo = (Map<String, Object>)o;
+      String key = (String)propertyInfo.get("key");
+      String val = (String)propertyInfo.get("value");
+      String resource = (String)propertyInfo.get("resource");
+      System.err.println("k: " + key + " v: " + val + " r: " + resource);
+      if (TEST_KEY.equals(key) && TEST_VAL.equals(val)
+          && "programatically".equals(resource)) {
+        foundSetting = true;
+      }
+    }
+    assertTrue(foundSetting);
+  }
+
+  @Test
+  public void testWriteXml() throws Exception {
+    StringWriter sw = new StringWriter();
+    ConfServlet.writeResponse(getTestConf(), sw, "xml");
+    String xml = sw.toString();
+
+    DocumentBuilderFactory docBuilderFactory 
+      = DocumentBuilderFactory.newInstance();
+    DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
+    Document doc = builder.parse(new InputSource(new StringReader(xml)));
+    NodeList nameNodes = doc.getElementsByTagName("name");
+    boolean foundSetting = false;
+    for (int i = 0; i < nameNodes.getLength(); i++) {
+      Node nameNode = nameNodes.item(i);
+      String key = nameNode.getTextContent();
+      System.err.println("xml key: " + key);
+      if (TEST_KEY.equals(key)) {
+        foundSetting = true;
+        Element propertyElem = (Element)nameNode.getParentNode();
+        String val = propertyElem.getElementsByTagName("value").item(0).getTextContent();
+        assertEquals(TEST_VAL, val);
+      }
+    }
+    assertTrue(foundSetting);
+  }
+
+  @Test
+  public void testBadFormat() throws Exception {
+    StringWriter sw = new StringWriter();
+    try {
+      ConfServlet.writeResponse(getTestConf(), sw, "not a format");
+      fail("writeResponse with bad format didn't throw!");
+    } catch (ConfServlet.BadFormatException bfe) {
+      // expected
+    }
+    assertEquals("", sw.toString());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
new file mode 100644
index 0000000..b95feba
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
@@ -0,0 +1,108 @@
+/*
+ * 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.net.URL;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.apache.hadoop.hbase.http.HttpServerFunctionalTest;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestJMXJsonServlet extends HttpServerFunctionalTest {
+  private   static final Log LOG = LogFactory.getLog(TestJMXJsonServlet.class);
+  private static HttpServer server;
+  private static URL baseUrl;
+
+  @BeforeClass public static void setup() throws Exception {
+    server = createTestServer();
+    server.start();
+    baseUrl = getServerURL(server);
+  }
+  
+  @AfterClass public static void cleanup() throws Exception {
+    server.stop();
+  }
+  
+  public static void assertReFind(String re, String value) {
+    Pattern p = Pattern.compile(re);
+    Matcher m = p.matcher(value);
+    assertTrue("'"+p+"' does not match "+value, m.find());
+  }
+  
+  @Test public void testQuery() throws Exception {
+    String result = readOutput(new URL(baseUrl, "/jmx?qry=java.lang:type=Runtime"));
+    LOG.info("/jmx?qry=java.lang:type=Runtime RESULT: "+result);
+    assertReFind("\"name\"\\s*:\\s*\"java.lang:type=Runtime\"", result);
+    assertReFind("\"modelerType\"", result);
+    
+    result = readOutput(new URL(baseUrl, "/jmx?qry=java.lang:type=Memory"));
+    LOG.info("/jmx?qry=java.lang:type=Memory RESULT: "+result);
+    assertReFind("\"name\"\\s*:\\s*\"java.lang:type=Memory\"", result);
+    assertReFind("\"modelerType\"", result);
+    
+    result = readOutput(new URL(baseUrl, "/jmx"));
+    LOG.info("/jmx RESULT: "+result);
+    assertReFind("\"name\"\\s*:\\s*\"java.lang:type=Memory\"", result);
+    
+    // test to get an attribute of a mbean
+    result = readOutput(new URL(baseUrl, 
+        "/jmx?get=java.lang:type=Memory::HeapMemoryUsage"));
+    LOG.info("/jmx RESULT: "+result);
+    assertReFind("\"name\"\\s*:\\s*\"java.lang:type=Memory\"", result);
+    assertReFind("\"committed\"\\s*:", result);
+    
+    // negative test to get an attribute of a mbean
+    result = readOutput(new URL(baseUrl, 
+        "/jmx?get=java.lang:type=Memory::"));
+    LOG.info("/jmx RESULT: "+result);
+    assertReFind("\"ERROR\"", result);
+
+    // test to get JSONP result
+    result = readOutput(new URL(baseUrl, "/jmx?qry=java.lang:type=Memory&callback=mycallback1"));
+    LOG.info("/jmx?qry=java.lang:type=Memory&callback=mycallback RESULT: "+result);
+    assertReFind("^mycallback1\\(\\{", result);
+    assertReFind("\\}\\);$", result);
+
+    // negative test to get an attribute of a mbean as JSONP
+    result = readOutput(new URL(baseUrl,
+        "/jmx?get=java.lang:type=Memory::&callback=mycallback2"));
+    LOG.info("/jmx RESULT: "+result);
+    assertReFind("^mycallback2\\(\\{", result);
+    assertReFind("\"ERROR\"", result);
+    assertReFind("\\}\\);$", result);
+
+    // test to get an attribute of a mbean as JSONP
+    result = readOutput(new URL(baseUrl,
+        "/jmx?get=java.lang:type=Memory::HeapMemoryUsage&callback=mycallback3"));
+    LOG.info("/jmx RESULT: "+result);
+    assertReFind("^mycallback3\\(\\{", result);
+    assertReFind("\"name\"\\s*:\\s*\"java.lang:type=Memory\"", result);
+    assertReFind("\"committed\"\\s*:", result);
+    assertReFind("\\}\\);$", result);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java
new file mode 100644
index 0000000..998c852
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java
@@ -0,0 +1,85 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.http.ServerConfigurationKeys;
+import org.apache.hadoop.hbase.http.lib.StaticUserWebFilter.StaticUserFilter;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+@Category(SmallTests.class)
+public class TestStaticUserWebFilter {
+  private FilterConfig mockConfig(String username) {
+    FilterConfig mock = Mockito.mock(FilterConfig.class);
+    Mockito.doReturn(username).when(mock).getInitParameter(
+            ServerConfigurationKeys.HBASE_HTTP_STATIC_USER);
+    return mock;
+  }
+  
+  @Test
+  public void testFilter() throws Exception {
+    FilterConfig config = mockConfig("myuser");
+    StaticUserFilter suf = new StaticUserFilter();
+    suf.init(config);
+    
+    ArgumentCaptor<HttpServletRequestWrapper> wrapperArg =
+      ArgumentCaptor.forClass(HttpServletRequestWrapper.class);
+
+    FilterChain chain = mock(FilterChain.class);
+    
+    suf.doFilter(mock(HttpServletRequest.class), mock(ServletResponse.class),
+        chain);
+        
+    Mockito.verify(chain).doFilter(wrapperArg.capture(), Mockito.<ServletResponse>anyObject());
+    
+    HttpServletRequestWrapper wrapper = wrapperArg.getValue();
+    assertEquals("myuser", wrapper.getUserPrincipal().getName());
+    assertEquals("myuser", wrapper.getRemoteUser());
+    
+    suf.destroy();
+  }
+  
+  @Test
+  public void testOldStyleConfiguration() {
+    Configuration conf = new Configuration();
+    conf.set("dfs.web.ugi", "joe,group1,group2");
+    assertEquals("joe", StaticUserWebFilter.getUsernameFromConf(conf));
+  }
+
+  @Test
+  public void testConfiguration() {
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER, "dr.stack");
+    assertEquals("dr.stack", StaticUserWebFilter.getUsernameFromConf(conf));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
new file mode 100644
index 0000000..3ca23d0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
@@ -0,0 +1,85 @@
+/**
+* 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.*;
+import java.net.*;
+
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.apache.hadoop.hbase.http.log.LogLevel;
+import org.apache.hadoop.net.NetUtils;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.*;
+import org.apache.commons.logging.impl.*;
+import org.apache.log4j.*;
+
+public class TestLogLevel extends TestCase {
+  static final PrintStream out = System.out;
+
+  public void testDynamicLogLevel() throws Exception {
+    String logName = TestLogLevel.class.getName();
+    Log testlog = LogFactory.getLog(logName);
+
+    //only test Log4JLogger
+    if (testlog instanceof Log4JLogger) {
+      Logger log = ((Log4JLogger)testlog).getLogger();
+      log.debug("log.debug1");
+      log.info("log.info1");
+      log.error("log.error1");
+      assertTrue(!Level.ERROR.equals(log.getEffectiveLevel()));
+
+      HttpServer server = new HttpServer.Builder().setName("..")
+          .addEndpoint(new URI("http://localhost:0")).setFindPort(true)
+          .build();
+      
+      server.start();
+      String authority = NetUtils.getHostPortString(server
+          .getConnectorAddress(0));
+
+      //servlet
+      URL url = new URL("http://" + authority + "/logLevel?log=" + logName
+          + "&level=" + Level.ERROR);
+      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; out.println(line));
+      in.close();
+
+      log.debug("log.debug2");
+      log.info("log.info2");
+      log.error("log.error2");
+      assertTrue(Level.ERROR.equals(log.getEffectiveLevel()));
+
+      //command line
+      String[] args = {"-setlevel", authority, logName, Level.DEBUG.toString()};
+      LogLevel.main(args);
+      log.debug("log.debug3");
+      log.info("log.info3");
+      log.error("log.error3");
+      assertTrue(Level.DEBUG.equals(log.getEffectiveLevel()));
+    }
+    else {
+      out.println(testlog.getClass() + " not tested.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/resource/JerseyResource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/resource/JerseyResource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/resource/JerseyResource.java
new file mode 100644
index 0000000..e38ae0f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/resource/JerseyResource.java
@@ -0,0 +1,64 @@
+/**
+ * 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.resource;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.mortbay.util.ajax.JSON;
+
+/**
+ * A simple Jersey resource class TestHttpServer.
+ * The servlet simply puts the path and the op parameter in a map
+ * and return it in JSON format in the response.
+ */
+@Path("")
+public class JerseyResource {
+  static final Log LOG = LogFactory.getLog(JerseyResource.class);
+
+  public static final String PATH = "path";
+  public static final String OP = "op";
+
+  @GET
+  @Path("{" + PATH + ":.*}")
+  @Produces({MediaType.APPLICATION_JSON})
+  public Response get(
+      @PathParam(PATH) @DefaultValue("UNKNOWN_" + PATH) final String path,
+      @QueryParam(OP) @DefaultValue("UNKNOWN_" + OP) final String op
+      ) throws IOException {
+    LOG.info("get: " + PATH + "=" + path + ", " + OP + "=" + op);
+
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put(PATH, path);
+    m.put(OP, op);
+    final String js = JSON.toString(m);
+    return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.java
new file mode 100644
index 0000000..248b820
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.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.http.ssl;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.Writer;
+import java.math.BigInteger;
+import java.net.URL;
+import java.security.GeneralSecurityException;
+import java.security.Key;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivateKey;
+import java.security.SecureRandom;
+import java.security.cert.Certificate;
+import java.security.cert.X509Certificate;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory;
+import org.apache.hadoop.security.ssl.SSLFactory;
+
+import sun.security.x509.AlgorithmId;
+import sun.security.x509.CertificateAlgorithmId;
+import sun.security.x509.CertificateIssuerName;
+import sun.security.x509.CertificateSerialNumber;
+import sun.security.x509.CertificateSubjectName;
+import sun.security.x509.CertificateValidity;
+import sun.security.x509.CertificateVersion;
+import sun.security.x509.CertificateX509Key;
+import sun.security.x509.X500Name;
+import sun.security.x509.X509CertImpl;
+import sun.security.x509.X509CertInfo;
+
+public class KeyStoreTestUtil {
+
+  public static String getClasspathDir(Class klass) throws Exception {
+    String file = klass.getName();
+    file = file.replace('.', '/') + ".class";
+    URL url = Thread.currentThread().getContextClassLoader().getResource(file);
+    String baseDir = url.toURI().getPath();
+    baseDir = baseDir.substring(0, baseDir.length() - file.length() - 1);
+    return baseDir;
+  }
+
+  /**
+   * Create a self-signed X.509 Certificate.
+   * From http://bfo.com/blog/2011/03/08/odds_and_ends_creating_a_new_x_509_certificate.html.
+   *
+   * @param dn the X.509 Distinguished Name, eg "CN=Test, L=London, C=GB"
+   * @param pair the KeyPair
+   * @param days how many days from now the Certificate is valid for
+   * @param algorithm the signing algorithm, eg "SHA1withRSA"
+   * @return the self-signed certificate
+   * @throws IOException thrown if an IO error ocurred.
+   * @throws GeneralSecurityException thrown if an Security error ocurred.
+   */
+  public static X509Certificate generateCertificate(String dn, KeyPair pair,
+                                                    int days, String algorithm)
+    throws GeneralSecurityException, IOException {
+    PrivateKey privkey = pair.getPrivate();
+    X509CertInfo info = new X509CertInfo();
+    Date from = new Date();
+    Date to = new Date(from.getTime() + days * 86400000l);
+    CertificateValidity interval = new CertificateValidity(from, to);
+    BigInteger sn = new BigInteger(64, new SecureRandom());
+    X500Name owner = new X500Name(dn);
+
+    info.set(X509CertInfo.VALIDITY, interval);
+    info.set(X509CertInfo.SERIAL_NUMBER, new CertificateSerialNumber(sn));
+    info.set(X509CertInfo.SUBJECT, new CertificateSubjectName(owner));
+    info.set(X509CertInfo.ISSUER, new CertificateIssuerName(owner));
+    info.set(X509CertInfo.KEY, new CertificateX509Key(pair.getPublic()));
+    info
+      .set(X509CertInfo.VERSION, new CertificateVersion(CertificateVersion.V3));
+    AlgorithmId algo = new AlgorithmId(AlgorithmId.md5WithRSAEncryption_oid);
+    info.set(X509CertInfo.ALGORITHM_ID, new CertificateAlgorithmId(algo));
+
+    // Sign the cert to identify the algorithm that's used.
+    X509CertImpl cert = new X509CertImpl(info);
+    cert.sign(privkey, algorithm);
+
+    // Update the algorith, and resign.
+    algo = (AlgorithmId) cert.get(X509CertImpl.SIG_ALG);
+    info
+      .set(CertificateAlgorithmId.NAME + "." + CertificateAlgorithmId.ALGORITHM,
+           algo);
+    cert = new X509CertImpl(info);
+    cert.sign(privkey, algorithm);
+    return cert;
+  }
+
+  public static KeyPair generateKeyPair(String algorithm)
+    throws NoSuchAlgorithmException {
+    KeyPairGenerator keyGen = KeyPairGenerator.getInstance(algorithm);
+    keyGen.initialize(1024);
+    return keyGen.genKeyPair();
+  }
+
+  private static KeyStore createEmptyKeyStore()
+    throws GeneralSecurityException, IOException {
+    KeyStore ks = KeyStore.getInstance("JKS");
+    ks.load(null, null); // initialize
+    return ks;
+  }
+
+  private static void saveKeyStore(KeyStore ks, String filename,
+                                   String password)
+    throws GeneralSecurityException, IOException {
+    FileOutputStream out = new FileOutputStream(filename);
+    try {
+      ks.store(out, password.toCharArray());
+    } finally {
+      out.close();
+    }
+  }
+
+  public static void createKeyStore(String filename,
+                                    String password, String alias,
+                                    Key privateKey, Certificate cert)
+    throws GeneralSecurityException, IOException {
+    KeyStore ks = createEmptyKeyStore();
+    ks.setKeyEntry(alias, privateKey, password.toCharArray(),
+                   new Certificate[]{cert});
+    saveKeyStore(ks, filename, password);
+  }
+
+  /**
+   * Creates a keystore with a single key and saves it to a file.
+   * 
+   * @param filename String file to save
+   * @param password String store password to set on keystore
+   * @param keyPassword String key password to set on key
+   * @param alias String alias to use for the key
+   * @param privateKey Key to save in keystore
+   * @param cert Certificate to use as certificate chain associated to key
+   * @throws GeneralSecurityException for any error with the security APIs
+   * @throws IOException if there is an I/O error saving the file
+   */
+  public static void createKeyStore(String filename,
+                                    String password, String keyPassword, String alias,
+                                    Key privateKey, Certificate cert)
+    throws GeneralSecurityException, IOException {
+    KeyStore ks = createEmptyKeyStore();
+    ks.setKeyEntry(alias, privateKey, keyPassword.toCharArray(),
+                   new Certificate[]{cert});
+    saveKeyStore(ks, filename, password);
+  }
+
+  public static void createTrustStore(String filename,
+                                      String password, String alias,
+                                      Certificate cert)
+    throws GeneralSecurityException, IOException {
+    KeyStore ks = createEmptyKeyStore();
+    ks.setCertificateEntry(alias, cert);
+    saveKeyStore(ks, filename, password);
+  }
+
+  public static <T extends Certificate> void createTrustStore(
+    String filename, String password, Map<String, T> certs)
+    throws GeneralSecurityException, IOException {
+    KeyStore ks = createEmptyKeyStore();
+    for (Map.Entry<String, T> cert : certs.entrySet()) {
+      ks.setCertificateEntry(cert.getKey(), cert.getValue());
+    }
+    saveKeyStore(ks, filename, password);
+  }
+
+  public static void cleanupSSLConfig(String keystoresDir, String sslConfDir)
+    throws Exception {
+    File f = new File(keystoresDir + "/clientKS.jks");
+    f.delete();
+    f = new File(keystoresDir + "/serverKS.jks");
+    f.delete();
+    f = new File(keystoresDir + "/trustKS.jks");
+    f.delete();
+    f = new File(sslConfDir + "/ssl-client.xml");
+    f.delete();
+    f = new File(sslConfDir +  "/ssl-server.xml");
+    f.delete();
+  }
+
+  /**
+   * Performs complete setup of SSL configuration in preparation for testing an
+   * SSLFactory.  This includes keys, certs, keystores, truststores, the server
+   * SSL configuration file, the client SSL configuration file, and the master
+   * configuration file read by the SSLFactory.
+   * 
+   * @param keystoresDir String directory to save keystores
+   * @param sslConfDir String directory to save SSL configuration files
+   * @param conf Configuration master configuration to be used by an SSLFactory,
+   *   which will be mutated by this method
+   * @param useClientCert boolean true to make the client present a cert in the
+   *   SSL handshake
+   */
+  public static void setupSSLConfig(String keystoresDir, String sslConfDir,
+                                    Configuration conf, boolean useClientCert)
+    throws Exception {
+    String clientKS = keystoresDir + "/clientKS.jks";
+    String clientPassword = "clientP";
+    String serverKS = keystoresDir + "/serverKS.jks";
+    String serverPassword = "serverP";
+    String trustKS = keystoresDir + "/trustKS.jks";
+    String trustPassword = "trustP";
+
+    File sslClientConfFile = new File(sslConfDir + "/ssl-client.xml");
+    File sslServerConfFile = new File(sslConfDir + "/ssl-server.xml");
+
+    Map<String, X509Certificate> certs = new HashMap<String, X509Certificate>();
+
+    if (useClientCert) {
+      KeyPair cKP = KeyStoreTestUtil.generateKeyPair("RSA");
+      X509Certificate cCert =
+        KeyStoreTestUtil.generateCertificate("CN=localhost, O=client", cKP, 30,
+                                             "SHA1withRSA");
+      KeyStoreTestUtil.createKeyStore(clientKS, clientPassword, "client",
+                                      cKP.getPrivate(), cCert);
+      certs.put("client", cCert);
+    }
+
+    KeyPair sKP = KeyStoreTestUtil.generateKeyPair("RSA");
+    X509Certificate sCert =
+      KeyStoreTestUtil.generateCertificate("CN=localhost, O=server", sKP, 30,
+                                           "SHA1withRSA");
+    KeyStoreTestUtil.createKeyStore(serverKS, serverPassword, "server",
+                                    sKP.getPrivate(), sCert);
+    certs.put("server", sCert);
+
+    KeyStoreTestUtil.createTrustStore(trustKS, trustPassword, certs);
+
+    Configuration clientSSLConf = createClientSSLConfig(clientKS, clientPassword,
+      clientPassword, trustKS);
+    Configuration serverSSLConf = createServerSSLConfig(serverKS, serverPassword,
+      serverPassword, trustKS);
+
+    saveConfig(sslClientConfFile, clientSSLConf);
+    saveConfig(sslServerConfFile, serverSSLConf);
+
+    conf.set(SSLFactory.SSL_HOSTNAME_VERIFIER_KEY, "ALLOW_ALL");
+    conf.set(SSLFactory.SSL_CLIENT_CONF_KEY, sslClientConfFile.getName());
+    conf.set(SSLFactory.SSL_SERVER_CONF_KEY, sslServerConfFile.getName());
+    conf.setBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY, useClientCert);
+  }
+
+  /**
+   * Creates SSL configuration for a client.
+   * 
+   * @param clientKS String client keystore file
+   * @param password String store password, or null to avoid setting store
+   *   password
+   * @param keyPassword String key password, or null to avoid setting key
+   *   password
+   * @param trustKS String truststore file
+   * @return Configuration for client SSL
+   */
+  public static Configuration createClientSSLConfig(String clientKS,
+      String password, String keyPassword, String trustKS) {
+    Configuration clientSSLConf = createSSLConfig(SSLFactory.Mode.CLIENT,
+      clientKS, password, keyPassword, trustKS);
+    return clientSSLConf;
+  }
+
+  /**
+   * Creates SSL configuration for a server.
+   * 
+   * @param serverKS String server keystore file
+   * @param password String store password, or null to avoid setting store
+   *   password
+   * @param keyPassword String key password, or null to avoid setting key
+   *   password
+   * @param trustKS String truststore file
+   * @return Configuration for server SSL
+   */
+  public static Configuration createServerSSLConfig(String serverKS,
+      String password, String keyPassword, String trustKS) throws IOException {
+    Configuration serverSSLConf = createSSLConfig(SSLFactory.Mode.SERVER,
+      serverKS, password, keyPassword, trustKS);
+    return serverSSLConf;
+  }
+
+  /**
+   * Creates SSL configuration.
+   * 
+   * @param mode SSLFactory.Mode mode to configure
+   * @param keystore String keystore file
+   * @param password String store password, or null to avoid setting store
+   *   password
+   * @param keyPassword String key password, or null to avoid setting key
+   *   password
+   * @param trustKS String truststore file
+   * @return Configuration for SSL
+   */
+  private static Configuration createSSLConfig(SSLFactory.Mode mode,
+      String keystore, String password, String keyPassword, String trustKS) {
+    String trustPassword = "trustP";
+
+    Configuration sslConf = new Configuration(false);
+    if (keystore != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_KEYSTORE_LOCATION_TPL_KEY), keystore);
+    }
+    if (password != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_KEYSTORE_PASSWORD_TPL_KEY), password);
+    }
+    if (keyPassword != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_KEYSTORE_KEYPASSWORD_TPL_KEY),
+        keyPassword);
+    }
+    if (trustKS != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_TRUSTSTORE_LOCATION_TPL_KEY), trustKS);
+    }
+    if (trustPassword != null) {
+      sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+        FileBasedKeyStoresFactory.SSL_TRUSTSTORE_PASSWORD_TPL_KEY),
+        trustPassword);
+    }
+    sslConf.set(FileBasedKeyStoresFactory.resolvePropertyName(mode,
+      FileBasedKeyStoresFactory.SSL_TRUSTSTORE_RELOAD_INTERVAL_TPL_KEY), "1000");
+
+    return sslConf;
+  }
+
+  /**
+   * Saves configuration to a file.
+   * 
+   * @param file File to save
+   * @param conf Configuration contents to write to file
+   * @throws IOException if there is an I/O error saving the file
+   */
+  public static void saveConfig(File file, Configuration conf)
+      throws IOException {
+    Writer writer = new FileWriter(file);
+    try {
+      conf.writeXml(writer);
+    } finally {
+      writer.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/26096f36/hbase-server/src/test/resources/webapps/static/test.css
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/resources/webapps/static/test.css b/hbase-server/src/test/resources/webapps/static/test.css
new file mode 100644
index 0000000..ae43828
--- /dev/null
+++ b/hbase-server/src/test/resources/webapps/static/test.css
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Test CSS file for content type handling - empty, since we just check
+ * returned content type!
+ */